diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java index 7f8ac3b8c9b4f..a21ba720ced04 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/CalciteQueryProcessor.java @@ -58,7 +58,9 @@ import org.apache.ignite.events.SqlQueryExecutionEvent; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.GridProcessorAdapter; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata; import org.apache.ignite.internal.processors.query.IgniteSQLException; @@ -120,6 +122,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.IgniteSystemProperties.getLong; +import static org.apache.ignite.configuration.TransactionConfiguration.TX_AWARE_QUERIES_SUPPORTED_MODES; import static org.apache.ignite.events.EventType.EVT_SQL_QUERY_EXECUTION; /** */ @@ -486,6 +489,8 @@ private List parseAndProcessQuery( String sql, Object... params ) throws IgniteSQLException { + ensureTransactionModeSupported(qryCtx); + SchemaPlus schema = schemaHolder.schema(schemaName); assert schema != null : "Schema not found: " + schemaName; @@ -588,7 +593,32 @@ private Object contextKey(QueryContext qryCtx) { SqlFieldsQuery sqlFieldsQry = qryCtx.unwrap(SqlFieldsQuery.class); - return sqlFieldsQry != null ? F.asList(sqlFieldsQry.isLocal(), sqlFieldsQry.isEnforceJoinOrder()) : null; + return sqlFieldsQry != null + ? F.asList(sqlFieldsQry.isLocal(), sqlFieldsQry.isEnforceJoinOrder(), queryTransactionVersion(qryCtx) == null) + : null; + } + + /** */ + private static GridCacheVersion queryTransactionVersion(@Nullable QueryContext qryCtx) { + return qryCtx == null ? null : qryCtx.unwrap(GridCacheVersion.class); + } + + /** */ + private void ensureTransactionModeSupported(@Nullable QueryContext qryCtx) { + if (!ctx.config().getTransactionConfiguration().isTxAwareQueriesEnabled()) + return; + + GridCacheVersion ver = queryTransactionVersion(qryCtx); + + if (ver == null) + return; + + final GridNearTxLocal userTx = ctx.cache().context().tm().tx(ver); + + if (TX_AWARE_QUERIES_SUPPORTED_MODES.contains(userTx.isolation())) + return; + + throw new IllegalStateException("Transaction isolation mode not supported for SQL queries: " + userTx.isolation()); } /** */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java index 6e87a7b3e6fb3..28b40966d94f7 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/AbstractIndexScan.java @@ -66,14 +66,14 @@ protected AbstractIndexScan( /** {@inheritDoc} */ @Override public synchronized Iterator iterator() { if (ranges == null) - return new IteratorImpl(idx.find(null, null, true, true, indexQueryContext())); + return new IteratorImpl(indexCursor(null, null, true, true)); IgniteClosure, IteratorImpl> clo = range -> { IdxRow lower = range.lower() == null ? null : row2indexRow(range.lower()); IdxRow upper = range.upper() == null ? null : row2indexRow(range.upper()); return new IteratorImpl( - idx.find(lower, upper, range.lowerInclude(), range.upperInclude(), indexQueryContext())); + indexCursor(lower, upper, range.lowerInclude(), range.upperInclude())); }; if (!ranges.multiBounds()) { @@ -88,6 +88,11 @@ protected AbstractIndexScan( return F.flat(F.iterator(ranges, clo, true)); } + /** */ + protected GridCursor indexCursor(IdxRow lower, IdxRow upper, boolean lowerInclude, boolean upperInclude) { + return idx.find(lower, upper, lowerInclude, upperInclude, indexQueryContext()); + } + /** */ protected abstract IdxRow row2indexRow(Row bound); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java index b10b69f221305..616ef7143091d 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExchangeServiceImpl.java @@ -361,6 +361,7 @@ private ExecutionContext baseInboxContext(UUID nodeId, UUID qryId, long fragm NoOpMemoryTracker.INSTANCE, NoOpIoTracker.INSTANCE, 0, - ImmutableMap.of()); + ImmutableMap.of(), + null); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java index 8627eb8e37cdf..2c8068e0316a6 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionContext.java @@ -18,24 +18,37 @@ package org.apache.ignite.internal.processors.query.calcite.exec; import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; +import java.util.function.Function; import org.apache.calcite.DataContext; import org.apache.calcite.linq4j.QueryProvider; import org.apache.calcite.schema.SchemaPlus; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactory; import org.apache.ignite.internal.processors.query.calcite.exec.exp.ExpressionFactoryImpl; import org.apache.ignite.internal.processors.query.calcite.exec.tracker.ExecutionNodeMemoryTracker; import org.apache.ignite.internal.processors.query.calcite.exec.tracker.IoTracker; import org.apache.ignite.internal.processors.query.calcite.exec.tracker.MemoryTracker; import org.apache.ignite.internal.processors.query.calcite.exec.tracker.RowTracker; +import org.apache.ignite.internal.processors.query.calcite.message.QueryTxEntry; import org.apache.ignite.internal.processors.query.calcite.metadata.ColocationGroup; import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription; import org.apache.ignite.internal.processors.query.calcite.prepare.AbstractQueryContext; @@ -45,8 +58,12 @@ import org.apache.ignite.internal.processors.query.calcite.util.Commons; import org.apache.ignite.internal.processors.query.calcite.util.TypeUtils; import org.apache.ignite.internal.util.lang.RunnableX; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.query.calcite.util.Commons.checkRange; @@ -102,6 +119,9 @@ public class ExecutionContext extends AbstractQueryContext implements DataC /** */ private final long timeout; + /** */ + private final Collection qryTxEntries; + /** */ private final long startTs; @@ -127,7 +147,8 @@ public ExecutionContext( MemoryTracker qryMemoryTracker, IoTracker ioTracker, long timeout, - Map params + Map params, + @Nullable Collection qryTxEntries ) { super(qctx); @@ -142,6 +163,7 @@ public ExecutionContext( this.ioTracker = ioTracker; this.params = params; this.timeout = timeout; + this.qryTxEntries = qryTxEntries; startTs = U.currentTimeMillis(); @@ -289,6 +311,84 @@ public void setCorrelated(@NotNull Object value, int id) { correlations[id] = value; } + /** + * @return Transaction write map. + */ + public Collection getQryTxEntries() { + return qryTxEntries; + } + + /** */ + public static Collection transactionChanges( + Collection writeEntries + ) { + if (F.isEmpty(writeEntries)) + return null; + + Collection res = new ArrayList<>(); + + for (IgniteTxEntry e : writeEntries) { + CacheObject val = e.value(); + + if (!F.isEmpty(e.entryProcessors())) + val = e.applyEntryProcessors(val); + + res.add(new QueryTxEntry(e.cacheId(), e.key(), val, e.explicitVersion())); + } + + return res; + } + + /** + * @param cacheId Cache id. + * @param parts Partitions set. + * @param mapper Mapper to specific data type. + * @return First, set of object changed in transaction, second, list of transaction data in required format. + * @param Required type. + */ + public IgniteBiTuple, List> transactionChanges( + int cacheId, + int[] parts, + Function mapper + ) { + if (F.isEmpty(qryTxEntries)) + return F.t(Collections.emptySet(), Collections.emptyList()); + + // Expecting parts are sorted or almost sorted and amount of transaction entries are relatively small. + if (parts != null && !F.isSorted(parts)) + Arrays.sort(parts); + + Set changedKeys = new HashSet<>(qryTxEntries.size()); + List newAndUpdatedRows = new ArrayList<>(qryTxEntries.size()); + + for (QueryTxEntry e : qryTxEntries) { + int part = e.key().partition(); + + assert part != -1; + + if (e.cacheId() != cacheId) + continue; + + if (parts != null && Arrays.binarySearch(parts, part) < 0) + continue; + + changedKeys.add(e.key()); + + CacheObject val = e.value(); + + if (val != null) { // Mix only updated or inserted entries. In case val == null entry removed. + newAndUpdatedRows.add(mapper.apply(new CacheDataRowAdapter( + e.key(), + val, + e.version(), + CU.EXPIRE_TIME_ETERNAL // Expire time calculated on commit, can use eternal here. + ))); + } + } + + return F.t(changedKeys, newAndUpdatedRows); + } + /** * Executes a query task. * diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java index bca13901aafd4..252358a870f52 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/ExecutionServiceImpl.java @@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.cache.CacheObjectValueContext; import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager; import org.apache.ignite.internal.processors.cache.QueryCursorImpl; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.query.CacheQueryType; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; @@ -608,6 +609,8 @@ private ListFieldsQueryCursor mapAndExecutePlan( MemoryTracker qryMemoryTracker = qry.createMemoryTracker(memoryTracker, cfg.getQueryMemoryQuota()); + final GridNearTxLocal userTx = Commons.queryTransaction(qry.context(), ctx.cache().context()); + ExecutionContext ectx = new ExecutionContext<>( qry.context(), taskExecutor(), @@ -620,7 +623,8 @@ private ListFieldsQueryCursor mapAndExecutePlan( qryMemoryTracker, createIoTracker(locNodeId, qry.localQueryId()), timeout, - qryParams); + qryParams, + userTx == null ? null : ExecutionContext.transactionChanges(userTx.writeEntries())); Node node = new LogicalRelImplementor<>(ectx, partitionService(), mailboxRegistry(), exchangeService(), failureProcessor()).go(fragment.root()); @@ -659,7 +663,8 @@ private ListFieldsQueryCursor mapAndExecutePlan( fragmentsPerNode.get(nodeId).intValue(), qry.parameters(), parametersMarshalled, - timeout + timeout, + ectx.getQryTxEntries() ); messageService().send(nodeId, req); @@ -867,7 +872,8 @@ private void onMessage(UUID nodeId, final QueryStartRequest msg) { qry.createMemoryTracker(memoryTracker, cfg.getQueryMemoryQuota()), createIoTracker(nodeId, msg.originatingQryId()), msg.timeout(), - Commons.parametersMap(msg.parameters()) + Commons.parametersMap(msg.parameters()), + msg.queryTransactionEntries() ); executeFragment(qry, (FragmentPlan)qryPlan, ectx); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java index d69d7c413342f..e30974f50d9a6 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/IndexScan.java @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Set; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.ImmutableIntList; @@ -31,15 +32,19 @@ import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyType; import org.apache.ignite.internal.cache.query.index.sorted.IndexPlainRowImpl; import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; +import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl; import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler; import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType; +import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexTree; +import org.apache.ignite.internal.cache.query.index.sorted.inline.SortedSegmentedIndexCursor; import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO; import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey; import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKeyFactory; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; @@ -54,6 +59,7 @@ import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl; import org.jetbrains.annotations.Nullable; @@ -98,6 +104,12 @@ public class IndexScan extends AbstractIndexScan { /** Types of key fields stored in index. */ private final Type[] fieldsStoreTypes; + /** + * First, set of keys changed (inserted, updated or removed) inside transaction: must be skiped during index scan. + * Second, list of rows inserted or updated inside transaction: must be mixed with the scan results. + */ + private final IgniteBiTuple, List> txChanges; + /** * @param ectx Execution context. * @param desc Table descriptor. @@ -159,6 +171,21 @@ protected IndexScan( fieldsStoreTypes[i] = typeFactory.getResultClass(srcRowType.getFieldList().get(i).getType()); fieldIdxMapping = fieldToInlinedKeysMapping(srcRowType.getFieldCount()); + + if (!F.isEmpty(ectx.getQryTxEntries())) { + InlineIndexRowHandler rowHnd = idx.segment(0).rowHandler(); + + txChanges = ectx.transactionChanges( + cctx.cacheId(), + parts, + r -> new IndexRowImpl(rowHnd, r) + ); + + txChanges.get2().sort(this::compare); + } + else + txChanges = null; + } /** @@ -221,6 +248,32 @@ private int[] fieldToInlinedKeysMapping(int srcFieldsCnt) { } } + /** {@inheritDoc} */ + @Override protected GridCursor indexCursor(IndexRow lower, IndexRow upper, boolean lowerInclude, boolean upperInclude) { + GridCursor idxCursor = super.indexCursor(lower, upper, lowerInclude, upperInclude); + + if (txChanges == null) + return idxCursor; + + // `txChanges` returns single thread data structures e.g. `HashSet`, `ArrayList`. + // It safe to use them in multiple `FilteredCursor` instances, because, multi range index scan will be flat to the single cursor. + // See AbstractIndexScan#iterator. + try { + return new SortedSegmentedIndexCursor( + new GridCursor[]{ + // This call will change `txChanges.get1()` content. + // Removing found key from set more efficient so we break some rules here. + new KeyFilteringCursor<>(idxCursor, txChanges.get1(), r -> r.cacheDataRow().key()), + new SortedListRangeCursor<>(this::compare, txChanges.get2(), lower, upper, lowerInclude, upperInclude) + }, + idx.indexDefinition() + ); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + /** {@inheritDoc} */ @Override protected IndexRow row2indexRow(Row bound) { if (bound == null) @@ -362,7 +415,7 @@ private synchronized void release() { InlineIndexRowHandler rowHnd = idx.segment(0).rowHandler(); - InlineIndexRowFactory rowFactory = isInlineScan() ? + InlineIndexRowFactory rowFactory = (isInlineScan() && (txChanges == null || F.isEmpty(txChanges.get1()))) ? new InlineIndexRowFactory(rowHnd.inlineIndexKeyTypes().toArray(new InlineIndexKeyType[0]), rowHnd) : null; BPlusTree.TreeRowClosure rowFilter = isInlineScan() ? null : createNotExpiredRowFilter(); @@ -444,6 +497,8 @@ public static BPlusTree.TreeRowClosure createNotNullRowFilte InlineIndexKeyType keyType = F.isEmpty(inlineKeyTypes) ? null : inlineKeyTypes.get(0); return new BPlusTree.TreeRowClosure() { + private IndexRow idxRow; + /** {@inheritDoc} */ @Override public boolean apply( BPlusTree tree, @@ -454,11 +509,14 @@ public static BPlusTree.TreeRowClosure createNotNullRowFilte if (!checkExpired && keyType != null && io instanceof InlineIO) { Boolean keyIsNull = keyType.isNull(pageAddr, io.offset(idx), ((InlineIO)io).inlineSize()); - if (keyIsNull == Boolean.TRUE) + if (keyIsNull == Boolean.TRUE) { + idxRow = null; + return false; + } } - IndexRow idxRow = io.getLookupRow(tree, pageAddr, idx); + idxRow = io.getLookupRow(tree, pageAddr, idx); if (checkExpired && idxRow.cacheDataRow().expireTime() > 0 && @@ -467,17 +525,35 @@ public static BPlusTree.TreeRowClosure createNotNullRowFilte return idxRow.key(0).type() != IndexKeyType.NULL; } + + /** {@inheritDoc} */ + @Override public IndexRow lastRow() { + return idxRow; + } }; } /** */ public static BPlusTree.TreeRowClosure createNotExpiredRowFilter() { - return (tree, io, pageAddr, idx) -> { - IndexRow idxRow = io.getLookupRow(tree, pageAddr, idx); + return new BPlusTree.TreeRowClosure() { + private IndexRow idxRow; + + @Override public boolean apply( + BPlusTree tree, + BPlusIO io, + long pageAddr, + int idx + ) throws IgniteCheckedException { + idxRow = io.getLookupRow(tree, pageAddr, idx); - // Skip expired. - return !(idxRow.cacheDataRow().expireTime() > 0 && - idxRow.cacheDataRow().expireTime() <= U.currentTimeMillis()); + // Skip expired. + return !(idxRow.cacheDataRow().expireTime() > 0 && + idxRow.cacheDataRow().expireTime() <= U.currentTimeMillis()); + } + + @Override public IndexRow lastRow() { + return idxRow; + } }; } @@ -507,4 +583,14 @@ protected TreeIndexWrapper(InlineIndex idx) { } } } + + /** */ + private int compare(IndexRow o1, IndexRow o2) { + try { + return InlineIndexTree.compareFullRows(o1, o2, 0, idx.segment(0).rowHandler(), idx.indexDefinition().rowComparator()); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursor.java new file mode 100644 index 0000000000000..4e0a0f44c50ad --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursor.java @@ -0,0 +1,73 @@ +/* + * 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.processors.query.calcite.exec; + +import java.util.Set; +import java.util.function.Function; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.util.lang.GridCursor; + +/** + * Cursor wrapper that skips all entires that maps to any of {@code skipKeys} key. + * Note, for the performance reasons content of {@code skipKeys} will be changed during iteration. + */ +class KeyFilteringCursor implements GridCursor { + /** Underlying cursor. */ + private final GridCursor cursor; + + /** Rows that must be skiped on {@link #cursor} iteration. */ + private final Set skipKeys; + + /** Mapper from row to {@link KeyCacheObject}. */ + private final Function toKey; + + /** + * @param cursor Sorted cursor. + * @param skipKeys Keys to skip. Content will be changed during iteration. + * @param toKey Mapper from row to {@link KeyCacheObject}. + */ + KeyFilteringCursor(GridCursor cursor, Set skipKeys, Function toKey) { + this.cursor = cursor; + this.skipKeys = skipKeys; + this.toKey = toKey; + } + + /** {@inheritDoc} */ + @Override public boolean next() throws IgniteCheckedException { + R cur; + + do { + if (!cursor.next()) + return false; + + cur = cursor.get(); + + // Intentionally use of `Set#remove` here. + // We want perform as few `toKey` as possible. + // So we break some rules here to optimize work with the data provided by the underlying cursor. + } while (!skipKeys.isEmpty() && skipKeys.remove(toKey.apply(cur))); + + return true; + } + + /** {@inheritDoc} */ + @Override public R get() throws IgniteCheckedException { + return cursor.get(); + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndex.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndex.java index ecb92c451c91f..c53a81e28af3a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndex.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndex.java @@ -18,7 +18,6 @@ import java.util.ArrayList; import java.util.Comparator; -import java.util.List; import java.util.Objects; import org.apache.calcite.rel.RelCollation; import org.apache.calcite.rel.type.RelDataType; @@ -26,7 +25,6 @@ import org.apache.ignite.internal.processors.query.calcite.exec.exp.RangeIterable; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.typedef.F; -import org.jetbrains.annotations.Nullable; /** * Runtime sorted index. @@ -90,7 +88,7 @@ public RuntimeSortedIndex( Row lowerRow = (lowerBound == null) ? null : lower; Row upperRow = (upperBound == null) ? null : upper; - return new Cursor(rows, lowerRow, upperRow, lowerInclude, upperInclude); + return new SortedListRangeCursor<>(comp, rows, lowerRow, upperRow, lowerInclude, upperInclude); } /** @@ -104,83 +102,6 @@ public Iterable scan( return new IndexScan(rowType, this, ranges); } - /** - * Cursor to navigate through a sorted list with duplicates. - */ - private class Cursor implements GridCursor { - /** List of rows. */ - private final List rows; - - /** Upper bound. */ - private final Row upper; - - /** Include upper bound. */ - private final boolean includeUpper; - - /** Current row. */ - private Row row; - - /** Current index of list element. */ - private int idx; - - /** - * @param rows List of rows. - * @param lower Lower bound. - * @param upper Upper bound. - * @param lowerInclude {@code True} for inclusive lower bound. - * @param upperInclude {@code True} for inclusive upper bound. - */ - Cursor(List rows, @Nullable Row lower, @Nullable Row upper, boolean lowerInclude, boolean upperInclude) { - this.rows = rows; - this.upper = upper; - this.includeUpper = upperInclude; - - idx = lower == null ? 0 : lowerBound(rows, lower, lowerInclude); - } - - /** - * Searches the lower bound (skipping duplicates) using a binary search. - * - * @param rows List of rows. - * @param bound Lower bound. - * @return Lower bound position in the list. - */ - private int lowerBound(List rows, Row bound, boolean includeBound) { - int low = 0, high = rows.size() - 1, idx = -1; - - while (low <= high) { - int mid = (high - low) / 2 + low; - int compRes = comp.compare(rows.get(mid), bound); - - if (compRes > 0) - high = mid - 1; - else if (compRes == 0 && includeBound) { - idx = mid; - high = mid - 1; - } - else - low = mid + 1; - } - - return idx == -1 ? low : idx; - } - - /** {@inheritDoc} */ - @Override public boolean next() { - if (idx == rows.size() || (upper != null && comp.compare(upper, rows.get(idx)) < (includeUpper ? 0 : 1))) - return false; - - row = rows.get(idx++); - - return true; - } - - /** {@inheritDoc} */ - @Override public Row get() { - return row; - } - } - /** * */ diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SortedListRangeCursor.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SortedListRangeCursor.java new file mode 100644 index 0000000000000..b56738b8edcb0 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/SortedListRangeCursor.java @@ -0,0 +1,115 @@ +/* + * 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.processors.query.calcite.exec; + +import java.util.Comparator; +import java.util.List; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.jetbrains.annotations.Nullable; + +/** + * Cursor to navigate through a sorted list with duplicates. + */ +public class SortedListRangeCursor implements GridCursor { + /** */ + private final Comparator comp; + + /** List of rows. */ + private final List rows; + + /** Upper bound. */ + private final Row upper; + + /** Include upper bound. */ + private final boolean includeUpper; + + /** Current row. */ + private Row row; + + /** Current index of list element. */ + private int idx; + + /** + * @param comp Rows comparator. + * @param rows List of rows. + * @param lower Lower bound. + * @param upper Upper bound. + * @param lowerInclude {@code True} for inclusive lower bound. + * @param upperInclude {@code True} for inclusive upper bound. + */ + public SortedListRangeCursor( + Comparator comp, + List rows, + @Nullable Row lower, + @Nullable Row upper, + boolean lowerInclude, + boolean upperInclude + ) { + this.comp = comp; + this.rows = rows; + this.upper = upper; + this.includeUpper = upperInclude; + + idx = lower == null ? 0 : lowerBound(rows, lower, lowerInclude); + } + + /** + * Searches the lower bound (skipping duplicates) using a binary search. + * + * @param rows List of rows. + * @param bound Lower bound. + * @return Lower bound position in the list. + */ + private int lowerBound(List rows, Row bound, boolean includeBound) { + int low = 0, high = rows.size() - 1, idx = -1; + + while (low <= high) { + int mid = (high - low) / 2 + low; + int compRes = comp.compare(rows.get(mid), bound); + + if (compRes > 0) + high = mid - 1; + else if (compRes == 0 && includeBound) { + idx = mid; + high = mid - 1; + } + else + low = mid + 1; + } + + return idx == -1 ? low : idx; + } + + /** {@inheritDoc} */ + @Override public boolean next() { + // Intentionally use `-comp.compare(rows.get(idx), upper)` here. + // `InlineIndexTree#compareFullRows` that used here works correctly only for specific parameter order. + // First parameter must be tree row and the second search row. See implementation, for details. + if (idx == rows.size() || (upper != null && -comp.compare(rows.get(idx), upper) < (includeUpper ? 0 : 1))) + return false; + + row = rows.get(idx++); + + return true; + } + + /** {@inheritDoc} */ + @Override public Row get() { + return row; + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java index f70d08231694b..2b618cf04d7e9 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/TableScan.java @@ -24,23 +24,28 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Queue; +import java.util.Set; +import java.util.function.Function; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.util.ImmutableBitSet; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterTopologyException; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheSearchRow; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory; import org.apache.ignite.internal.processors.query.calcite.schema.CacheTableDescriptor; import org.apache.ignite.internal.util.lang.GridCursor; import org.apache.ignite.internal.util.lang.GridIteratorAdapter; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; import org.jetbrains.annotations.Nullable; /** */ @@ -187,6 +192,15 @@ private class IteratorImpl extends GridIteratorAdapter { /** */ private GridCursor cur; + /** + * First, set of keys changed (inserted, updated or removed) inside transaction: must be skiped during index scan. + * Second, list of rows inserted or updated inside transaction: must be mixed with the scan results. + */ + private IgniteBiTuple, List> txChanges; + + /** */ + private Iterator txIter = Collections.emptyIterator(); + /** */ private Row next; @@ -195,6 +209,16 @@ private IteratorImpl() { assert reserved != null; parts = new ArrayDeque<>(reserved); + + if (!F.isEmpty(ectx.getQryTxEntries())) { + txChanges = ectx.transactionChanges( + cctx.cacheId(), + // All partitions scaned for replication cache. + // See TableScan#reserve. + cctx.isReplicated() ? null : TableScan.this.parts, + Function.identity() + ); + } } /** {@inheritDoc} */ @@ -237,11 +261,25 @@ private void advance() throws IgniteCheckedException { break; cur = part.dataStore().cursor(cctx.cacheId()); + + if (txChanges != null) { + // This call will change `txChanges.get1()` content. + // Removing found key from set more efficient so we break some rules here. + if (!F.isEmpty(txChanges.get1())) + cur = new KeyFilteringCursor<>(cur, txChanges.get1(), CacheSearchRow::key); + + txIter = F.iterator0(txChanges.get2(), true, e -> e.key().partition() == part.id()); + } } - if (cur.next()) { - CacheDataRow row = cur.get(); + CacheDataRow row; + + if (cur.next()) + row = cur.get(); + else + row = txIter.hasNext() ? txIter.next() : null; + if (row != null) { if (row.expireTime() > 0 && row.expireTime() <= U.currentTimeMillis()) continue; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java index 112fd30c24bdf..3b15b6831cc78 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/exp/ExpressionFactoryImpl.java @@ -62,6 +62,7 @@ import org.apache.calcite.rex.RexUtil; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlConformance; +import org.apache.ignite.internal.binary.BinaryObjectImpl; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler.RowFactory; @@ -246,11 +247,26 @@ else if (c2 != HIGHEST_VALUE) /** */ @SuppressWarnings("rawtypes") private static int compare(Object o1, Object o2, int nullComparison) { + if (o1 instanceof BinaryObjectImpl) + return compareBinary(o1, o2, nullComparison); + final Comparable c1 = (Comparable)o1; final Comparable c2 = (Comparable)o2; return RelFieldCollation.compare(c1, c2, nullComparison); } + /** */ + private static int compareBinary(Object o1, Object o2, int nullComparison) { + if (o1 == o2) + return 0; + else if (o1 == null) + return nullComparison; + else if (o2 == null) + return -nullComparison; + + return BinaryObjectImpl.compareForDml(o1, o2); + } + /** {@inheritDoc} */ @Override public Predicate predicate(RexNode filter, RelDataType rowType) { return new PredicateImpl(scalar(filter, rowType)); diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java index 4491028d0beb8..5a9f3dac4ef54 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/ModifyNode.java @@ -30,10 +30,13 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheProxyImpl; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; import org.apache.ignite.internal.processors.query.calcite.schema.CacheTableDescriptor; import org.apache.ignite.internal.processors.query.calcite.schema.ModifyTuple; +import org.apache.ignite.internal.processors.query.calcite.util.Commons; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -195,8 +198,27 @@ private void flushTuples(boolean force) throws IgniteCheckedException { this.tuples = new ArrayList<>(MODIFY_BATCH_SIZE); GridCacheContext cctx = desc.cacheContext(); + GridCacheProxyImpl cache = cctx.cache().keepBinary(); + GridNearTxLocal tx = Commons.queryTransaction(context(), cctx.shared()); + + if (tx == null) + invokeOutsideTransaction(tuples, cache); + else + invokeInsideTransaction(tuples, cache, tx); + } + + /** + * Perform data modification without explicit transaction. + * @param tuples Modified data. + * @param cache Cache. + * @throws IgniteCheckedException If failed. + */ + private void invokeOutsideTransaction( + List tuples, + GridCacheProxyImpl cache + ) throws IgniteCheckedException { Map> map = invokeMap(tuples); - Map> res = cctx.cache().keepBinary().invokeAll(map); + Map> res = cache.invokeAll(map); long updated = res.values().stream().mapToLong(EntryProcessorResult::get).sum(); @@ -212,6 +234,28 @@ private void flushTuples(boolean force) throws IgniteCheckedException { updatedRows += updated; } + /** + * Performs data modification within user transaction. + * @param tuples Modified data. + * @param cache Cache. + * @param userTx Transaction. + * @throws IgniteCheckedException If failed. + */ + private void invokeInsideTransaction( + List tuples, + GridCacheProxyImpl cache, + GridNearTxLocal userTx + ) throws IgniteCheckedException { + userTx.resume(); + + try { + invokeOutsideTransaction(tuples, cache); + } + finally { + userTx.suspend(); + } + } + /** */ private IgniteSQLException conflictKeysException(List conflictKeys) { if (op == TableModify.Operation.INSERT) { diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java index d32775364e982..56b57c385c725 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/MessageType.java @@ -57,7 +57,10 @@ public enum MessageType { COLOCATION_GROUP(351, ColocationGroup::new), /** */ - FRAGMENT_DESCRIPTION(352, FragmentDescription::new); + FRAGMENT_DESCRIPTION(352, FragmentDescription::new), + + /** */ + QUERY_TX_ENTRY(353, QueryTxEntry::new); /** */ private final int directType; diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java index b8d5eb8f8f351..4b63bf0256703 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryStartRequest.java @@ -18,13 +18,16 @@ package org.apache.ignite.internal.processors.query.calcite.message; import java.nio.ByteBuffer; +import java.util.Collection; import java.util.UUID; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridDirectCollection; import org.apache.ignite.internal.GridDirectTransient; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentDescription; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.jetbrains.annotations.Nullable; @@ -64,6 +67,10 @@ public class QueryStartRequest implements MarshalableMessage, ExecutionContextAw /** */ private long timeout; + /** */ + @GridDirectCollection(QueryTxEntry.class) + private @Nullable Collection qryTxEntries; + /** */ @SuppressWarnings("AssignmentOrReturnOfFieldWithMutableType") public QueryStartRequest( @@ -76,7 +83,8 @@ public QueryStartRequest( int totalFragmentsCnt, Object[] params, @Nullable byte[] paramsBytes, - long timeout + long timeout, + Collection qryTxEntries ) { this.qryId = qryId; this.originatingQryId = originatingQryId; @@ -88,6 +96,7 @@ public QueryStartRequest( this.params = params; this.paramsBytes = paramsBytes; // If we already have marshalled params, use it. this.timeout = timeout; + this.qryTxEntries = qryTxEntries; } /** */ @@ -168,20 +177,39 @@ public long timeout() { return timeout; } + /** + * @return Transaction entries to mixin on query processing. + */ + public @Nullable Collection queryTransactionEntries() { + return qryTxEntries; + } + /** {@inheritDoc} */ @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { if (paramsBytes == null && params != null) paramsBytes = U.marshal(ctx, params); fragmentDesc.prepareMarshal(ctx); + + if (qryTxEntries != null) { + for (QueryTxEntry e : qryTxEntries) + e.prepareMarshal(ctx); + } } /** {@inheritDoc} */ @Override public void prepareUnmarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + ClassLoader ldr = U.resolveClassLoader(ctx.gridConfig()); + if (params == null && paramsBytes != null) - params = U.unmarshal(ctx, paramsBytes, U.resolveClassLoader(ctx.gridConfig())); + params = U.unmarshal(ctx, paramsBytes, ldr); fragmentDesc.prepareUnmarshal(ctx); + + if (qryTxEntries != null) { + for (QueryTxEntry e : qryTxEntries) + e.prepareUnmarshal(ctx, ldr); + } } /** {@inheritDoc} */ @@ -245,6 +273,12 @@ public long timeout() { writer.incrementState(); case 8: + if (!writer.writeCollection("qryTxEntries", qryTxEntries, MessageCollectionItemType.MSG)) + return false; + + writer.incrementState(); + + case 9: if (!writer.writeAffinityTopologyVersion("ver", ver)) return false; @@ -328,6 +362,14 @@ public long timeout() { reader.incrementState(); case 8: + qryTxEntries = reader.readCollection("qryTxEntries", MessageCollectionItemType.MSG); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 9: ver = reader.readAffinityTopologyVersion("ver"); if (!reader.isLastRead()) @@ -347,6 +389,6 @@ public long timeout() { /** {@inheritDoc} */ @Override public byte fieldsCount() { - return 9; + return 10; } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryTxEntry.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryTxEntry.java new file mode 100644 index 0000000000000..6ee247e29f088 --- /dev/null +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/message/QueryTxEntry.java @@ -0,0 +1,211 @@ +/* + * 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.processors.query.calcite.message; + +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.function.Function; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.processors.cache.CacheObject; +import org.apache.ignite.internal.processors.cache.CacheObjectContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; +import org.apache.ignite.plugin.extensions.communication.MessageReader; +import org.apache.ignite.plugin.extensions.communication.MessageWriter; + +/** + * Class to pass to remote nodes transaction changes. + * + * @see TransactionConfiguration#setTxAwareQueriesEnabled(boolean) + * @see ExecutionContext#transactionChanges(Collection) + * @see ExecutionContext#transactionChanges(int, int[], Function) + * @see QueryStartRequest#queryTransactionEntries() + */ +public class QueryTxEntry implements CalciteMessage { + /** Cache id. */ + private int cacheId; + + /** Entry key. */ + private KeyCacheObject key; + + /** Entry value. */ + private CacheObject val; + + /** Entry version. */ + private GridCacheVersion ver; + + /** + * Empty constructor. + */ + public QueryTxEntry() { + // No-op. + } + + /** + * @param cacheId Cache id. + * @param key Key. + * @param val Value. + * @param ver Version. + */ + public QueryTxEntry(int cacheId, KeyCacheObject key, CacheObject val, GridCacheVersion ver) { + this.cacheId = cacheId; + this.key = key; + this.val = val; + this.ver = ver; + } + + /** @return Cache id. */ + public int cacheId() { + return cacheId; + } + + /** @return Entry key. */ + public KeyCacheObject key() { + return key; + } + + /** @return Entry value. */ + public CacheObject value() { + return val; + } + + /** @return Entry version. */ + public GridCacheVersion version() { + return ver; + } + + /** */ + public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException { + CacheObjectContext coctx = ctx.cacheContext(cacheId).cacheObjectContext(); + + key.prepareMarshal(coctx); + + if (val != null) + val.prepareMarshal(coctx); + } + + /** */ + public void prepareUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException { + CacheObjectContext coctx = ctx.cacheContext(cacheId).cacheObjectContext(); + + key.finishUnmarshal(coctx, ldr); + + if (val != null) + val.finishUnmarshal(coctx, ldr); + + } + + /** {@inheritDoc} */ + @Override public MessageType type() { + return MessageType.QUERY_TX_ENTRY; + } + + /** {@inheritDoc} */ + @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) { + writer.setBuffer(buf); + + if (!writer.isHeaderWritten()) { + if (!writer.writeHeader(directType(), fieldsCount())) + return false; + + writer.onHeaderWritten(); + } + + switch (writer.state()) { + case 0: + if (!writer.writeInt("cacheId", cacheId)) + return false; + + writer.incrementState(); + + case 1: + if (!writer.writeMessage("key", key)) + return false; + + writer.incrementState(); + + case 2: + if (!writer.writeMessage("val", val)) + return false; + + writer.incrementState(); + + case 3: + if (!writer.writeMessage("ver", ver)) + return false; + + writer.incrementState(); + + } + + return true; + } + + /** {@inheritDoc} */ + @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) { + reader.setBuffer(buf); + + if (!reader.beforeMessageRead()) + return false; + + switch (reader.state()) { + case 0: + cacheId = reader.readInt("cacheId"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 1: + key = reader.readMessage("key"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 2: + val = reader.readMessage("val"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + case 3: + ver = reader.readMessage("ver"); + + if (!reader.isLastRead()) + return false; + + reader.incrementState(); + + } + + return reader.afterMessageRead(QueryTxEntry.class); + } + + /** {@inheritDoc} */ + @Override public byte fieldsCount() { + return 4; + } +} diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryTemplate.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryTemplate.java index 1958488fcc574..4bef48c95505a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryTemplate.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/prepare/QueryTemplate.java @@ -18,6 +18,7 @@ package org.apache.ignite.internal.processors.query.calcite.prepare; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -25,14 +26,19 @@ import java.util.concurrent.atomic.AtomicReference; import com.google.common.collect.ImmutableList; +import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.metadata.RelMetadataQuery; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.calcite.exec.PartitionExtractor; import org.apache.ignite.internal.processors.query.calcite.exec.partition.PartitionNode; import org.apache.ignite.internal.processors.query.calcite.metadata.FragmentMappingException; import org.apache.ignite.internal.processors.query.calcite.metadata.MappingService; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteExchange; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteReceiver; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteRel; import org.apache.ignite.internal.processors.query.calcite.rel.IgniteSender; +import org.apache.ignite.internal.processors.query.calcite.rel.IgniteTableModify; +import org.apache.ignite.internal.processors.query.calcite.trait.IgniteDistributions; import org.apache.ignite.internal.processors.query.calcite.util.Commons; import org.apache.ignite.internal.util.typedef.F; import org.jetbrains.annotations.NotNull; @@ -85,7 +91,27 @@ public ExecutionPlan map(MappingService mappingService, MappingQueryContext ctx) else ex.addSuppressed(e); - fragments = replace(fragments, e.fragment(), new FragmentSplitter(e.node()).go(e.fragment())); + RelNode cutPoint = e.node(); + + if (Commons.queryTransactionVersion(ctx) != null && cutPoint instanceof IgniteTableModify) { + IgniteRel input = (IgniteRel)cutPoint.getInput(0); + + // Table modify under transaction should always be executed on initiator node. + if (!input.distribution().equals(IgniteDistributions.single())) { + cutPoint = ((IgniteRel)cutPoint).clone(cutPoint.getCluster(), Collections.singletonList( + new IgniteExchange( + input.getCluster(), + input.getTraitSet().replace(IgniteDistributions.single()), + input, + IgniteDistributions.single()))); + + fragments = replace(fragments, e.fragment(), new Splitter().go((IgniteRel)cutPoint)); + + continue; + } + } + + fragments = replace(fragments, e.fragment(), new FragmentSplitter(cutPoint).go(e.fragment())); } } diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheIndexImpl.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheIndexImpl.java index 00e4fa8480c54..aeed8641c010a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheIndexImpl.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/schema/CacheIndexImpl.java @@ -19,7 +19,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.UUID; +import java.util.function.Function; import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelCollation; @@ -35,11 +37,15 @@ import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyType; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; +import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler; import org.apache.ignite.internal.cache.query.index.sorted.inline.IndexQueryContext; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndex; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexImpl; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType; import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyTypeRegistry; +import org.apache.ignite.internal.cache.query.index.sorted.keys.NullIndexKey; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.persistence.tree.BPlusTree; import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; @@ -51,8 +57,11 @@ import org.apache.ignite.internal.processors.query.calcite.rel.logical.IgniteLogicalIndexScan; import org.apache.ignite.internal.processors.query.calcite.util.Commons; import org.apache.ignite.internal.processors.query.calcite.util.RexUtils; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.spi.indexing.IndexingQueryFilter; import org.apache.ignite.spi.indexing.IndexingQueryFilterImpl; +import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; /** @@ -155,16 +164,41 @@ public Index queryIndex() { if (idx == null || !grp.nodeIds().contains(ectx.localNodeId())) return 0; + int[] locParts = grp.partitions(ectx.localNodeId()); + InlineIndex iidx = idx.unwrap(InlineIndex.class); + boolean[] skipCheck = new boolean[] {false}; + + BPlusTree.TreeRowClosure rowFilter = countRowFilter(skipCheck, notNull, iidx); + + long cnt = 0; + + if (!F.isEmpty(ectx.getQryTxEntries())) { + IgniteBiTuple, List> txChanges = ectx.transactionChanges( + iidx.indexDefinition().cacheInfo().cacheId(), + locParts, + Function.identity() + ); + + if (!txChanges.get1().isEmpty()) { + // This call will change `txChanges.get1()` content. + // Removing found key from set more efficient so we break some rules here. + rowFilter = transactionAwareCountRowFilter(rowFilter, txChanges.get1()); + + cnt = countTransactionRows(notNull, iidx, txChanges.get2()); + } + } + try { IndexingQueryFilter filter = new IndexingQueryFilterImpl(tbl.descriptor().cacheContext().kernalContext(), - ectx.topologyVersion(), grp.partitions(ectx.localNodeId())); + ectx.topologyVersion(), locParts); - long cnt = 0; + for (int i = 0; i < iidx.segmentsCount(); ++i) { + cnt += iidx.count(i, new IndexQueryContext(filter, rowFilter)); - for (int i = 0; i < iidx.segmentsCount(); ++i) - cnt += iidx.count(i, new IndexQueryContext(filter, countRowFilter(notNull, iidx))); + skipCheck[0] = false; + } return cnt; } @@ -174,7 +208,7 @@ public Index queryIndex() { } /** */ - private @Nullable BPlusTree.TreeRowClosure countRowFilter(boolean notNull, InlineIndex iidx) { + private @Nullable BPlusTree.TreeRowClosure countRowFilter(boolean[] skipCheck, boolean notNull, InlineIndex iidx) { boolean checkExpired = !tbl.descriptor().cacheContext().config().isEagerTtl(); if (notNull) { @@ -183,8 +217,6 @@ public Index queryIndex() { BPlusTree.TreeRowClosure notNullRowFilter = IndexScan.createNotNullRowFilter(iidx, checkExpired); return new BPlusTree.TreeRowClosure<>() { - private boolean skipCheck; - @Override public boolean apply( BPlusTree tree, BPlusIO io, @@ -195,22 +227,72 @@ public Index queryIndex() { // don't need to check it with notNullRowFilter. // In case of NULL-LAST collation, all values after first null value will be null, // don't need to check it too. - if (skipCheck && !checkExpired) + if (skipCheck[0] && !checkExpired) return nullsFirst; boolean res = notNullRowFilter.apply(tree, io, pageAddr, idx); if (res == nullsFirst) - skipCheck = true; + skipCheck[0] = true; return res; } + + @Override public IndexRow lastRow() { + return (skipCheck[0] && !checkExpired) + ? null + : notNullRowFilter.lastRow(); + } }; } return checkExpired ? IndexScan.createNotExpiredRowFilter() : null; } + /** */ + private static @NotNull BPlusTree.TreeRowClosure transactionAwareCountRowFilter( + BPlusTree.TreeRowClosure rowFilter, + Set skipKeys + ) { + return new BPlusTree.TreeRowClosure<>() { + @Override public boolean apply( + BPlusTree tree, + BPlusIO io, + long pageAddr, + int idx + ) throws IgniteCheckedException { + if (rowFilter != null && !rowFilter.apply(tree, io, pageAddr, idx)) + return false; + + if (skipKeys.isEmpty()) + return true; + + IndexRow row = rowFilter == null ? null : rowFilter.lastRow(); + + if (row == null) + row = tree.getRow(io, pageAddr, idx); + + return !skipKeys.remove(row.cacheDataRow().key()); + } + }; + } + + /** */ + private static long countTransactionRows(boolean notNull, InlineIndex iidx, List changedRows) { + InlineIndexRowHandler rowHnd = iidx.segment(0).rowHandler(); + + long cnt = 0; + + for (CacheDataRow txRow : changedRows) { + if (rowHnd.indexKey(0, txRow) == NullIndexKey.INSTANCE && notNull) + continue; + + cnt++; + } + + return cnt; + } + /** {@inheritDoc} */ @Override public List toSearchBounds( RelOptCluster cluster, diff --git a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java index 468c4f53d7b46..c0fc06977160a 100644 --- a/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java +++ b/modules/calcite/src/main/java/org/apache/ignite/internal/processors/query/calcite/util/Commons.java @@ -60,7 +60,10 @@ import org.apache.ignite.internal.GridComponent; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryContext; import org.apache.ignite.internal.processors.query.calcite.exec.RowHandler; @@ -517,4 +520,23 @@ public static MappingQueryContext mapContext( ) { return new MappingQueryContext(ctx, locNodeId, topVer, qryParams); } + + /** + * @param ctx Query context. + * @param cctx Grid cache shared context. + * @return Query transaction or {@code null}. + */ + public static T queryTransaction(Context ctx, GridCacheSharedContext cctx) { + GridCacheVersion txId = queryTransactionVersion(ctx); + + return txId == null ? null : cctx.tm().tx(txId); + } + + /** + * @param ctx Context. + * @return Query transaction version if exists or {@code null}. + */ + public static @Nullable GridCacheVersion queryTransactionVersion(Context ctx) { + return ctx.unwrap(GridCacheVersion.class); + } } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java index 2d8f936919d18..00381ec93f03f 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/QueryChecker.java @@ -27,21 +27,24 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; - import org.apache.calcite.tools.FrameworkConfig; import org.apache.ignite.Ignite; import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; +import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.query.QueryContext; import org.apache.ignite.internal.processors.query.QueryEngine; +import org.apache.ignite.internal.processors.query.calcite.integration.AbstractBasicIntegrationTransactionalTest.SqlTransactionMode; import org.apache.ignite.internal.processors.query.schema.management.SchemaManager; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.transactions.Transaction; import org.hamcrest.CoreMatchers; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.Matcher; @@ -274,6 +277,9 @@ public static Matcher containsAnyScan(final String schema, final String /** */ private final String qry; + /** */ + private final Transaction tx; + /** */ private final ArrayList> planMatchers = new ArrayList<>(); @@ -303,7 +309,16 @@ public static Matcher containsAnyScan(final String schema, final String /** */ public QueryChecker(String qry) { + this(qry, null, SqlTransactionMode.NONE); + } + + /** */ + public QueryChecker(String qry, Transaction tx, SqlTransactionMode sqlTxMode) { + assert (tx != null && sqlTxMode != SqlTransactionMode.NONE) + || (tx == null && sqlTxMode == SqlTransactionMode.NONE) : "mode = " + sqlTxMode + ", tx = " + tx; + this.qry = qry; + this.tx = tx; } /** */ @@ -382,7 +397,11 @@ public void check() { // Check plan. QueryEngine engine = getEngine(); - QueryContext ctx = frameworkCfg != null ? QueryContext.of(frameworkCfg) : null; + GridCacheVersion txVer = tx != null + ? ((TransactionProxyImpl)tx).tx().xidVersion() + : null; + + QueryContext ctx = (frameworkCfg != null || txVer != null) ? QueryContext.of(frameworkCfg, txVer) : null; List>> explainCursors = engine.query(ctx, "PUBLIC", "EXPLAIN PLAN FOR " + qry, params); @@ -391,7 +410,8 @@ public void check() { List> explainRes = explainCursor.getAll(); String actualPlan = (String)explainRes.get(0).get(0); - if (!F.isEmpty(planMatchers)) { + // Will not check plan in transaction, because, statistic not refreshed inside transaction, so plan differs from expected. + if (!F.isEmpty(planMatchers) && tx == null) { for (Matcher matcher : planMatchers) assertThat("Invalid plan:\n" + actualPlan + "\n for query: " + qry, actualPlan, matcher); } @@ -520,7 +540,7 @@ public static void awaitReservationsRelease(String cacheName) throws IgniteInter * @param cacheName Cache to check reservations. */ public static void awaitReservationsRelease(IgniteEx node, String cacheName) throws IgniteInterruptedCheckedException { - GridDhtAtomicCache c = GridTestUtils.getFieldValue(node.cachex(cacheName), "delegate"); + GridDhtCacheAdapter c = GridTestUtils.getFieldValue(node.cachex(cacheName), "delegate"); List parts = c.topology().localPartitions(); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursorTest.java new file mode 100644 index 0000000000000..ff014e59f4e9c --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/KeyFilteringCursorTest.java @@ -0,0 +1,160 @@ +/* + * 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.processors.query.calcite.exec; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; +import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; +import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; +import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler; +import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType; +import org.apache.ignite.internal.cache.query.index.sorted.keys.IndexKey; +import org.apache.ignite.internal.cache.query.index.sorted.keys.IntegerIndexKey; +import org.apache.ignite.internal.processors.cache.KeyCacheObject; +import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; +import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.util.lang.GridCursor; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +/** */ +public class KeyFilteringCursorTest { + /** */ + @Test + public void testSimple() throws Exception { + assertEquals(Collections.emptyList(), all(cursor(), keys())); + + assertEquals(Arrays.asList(1, 3), all(cursor(1, 2, 3), keys(2))); + + assertEquals(Arrays.asList(1, 2, 3), all(cursor(1, 2, 3), keys())); + + assertEquals(Collections.emptyList(), all(cursor(1, 2, 3), keys(1, 2, 3))); + + assertEquals(Arrays.asList(1, 2, 3), all(cursor(1, 2, 3), keys(4, 5, 6))); + + assertEquals(Arrays.asList(2, 3), all(cursor(1, 2, 3), keys(1, 4, 5, 6))); + + assertEquals(Collections.emptyList(), all(cursor(1, 2, 3), keys(1, 2, 3, 4, 5, 6))); + + assertEquals(Collections.singletonList(2), all(cursor(1, 2, 3, 4, 5, 6), keys(1, 3, 4, 5, 6))); + + assertEquals(Arrays.asList(1, 2, 3, 4), all(cursor(1, 2, 3, 4, 5, 6), keys(5, 6))); + } + + /** */ + private List all(GridCursor rawCursor, Set skipKeys) throws IgniteCheckedException { + GridCursor cursor = new KeyFilteringCursor<>(rawCursor, skipKeys, r -> r.cacheDataRow().key()); + + List res = new ArrayList<>(); + + while (cursor.next()) + res.add((Integer)cursor.get().key(0).key()); + + return res; + } + + /** */ + private GridCursor cursor(int...nums) { + List rows = new ArrayList<>(); + + for (int num : nums) { + rows.add(new IndexRow() { + @Override public IndexKey key(int idx) { + return new IntegerIndexKey(num); + } + + @Override public int keysCount() { + return 1; + } + + @Override public long link() { + throw new UnsupportedOperationException(); + } + + @Override public InlineIndexRowHandler rowHandler() { + return new InlineIndexRowHandler() { + @Override public IndexKey indexKey(int idx, CacheDataRow row) { + assert idx == 0; + + return new IntegerIndexKey(num); + } + + @Override public List inlineIndexKeyTypes() { + throw new UnsupportedOperationException(); + } + + @Override public List indexKeyDefinitions() { + throw new UnsupportedOperationException(); + } + + @Override public IndexKeyTypeSettings indexKeyTypeSettings() { + throw new UnsupportedOperationException(); + } + + @Override public int partition(CacheDataRow row) { + return row.partition(); + } + + @Override public Object cacheKey(CacheDataRow row) { + return row.key(); + } + + @Override public Object cacheValue(CacheDataRow row) { + return row.value(); + } + }; + } + + @Override public CacheDataRow cacheDataRow() { + return new CacheDataRowAdapter( + KeyFilteringCursorTest.this.key(num), + null, + new GridCacheVersion(1, 1, 1L), + CU.EXPIRE_TIME_ETERNAL + ); + } + + @Override public boolean indexPlainRow() { + return false; + } + }); + } + + return new SortedListRangeCursor<>(null, rows, null, null, true, true); + } + + /** */ + private Set keys(int...nums) { + return Arrays.stream(nums).mapToObj(this::key).collect(Collectors.toSet()); + } + + /** */ + private KeyCacheObjectImpl key(int num) { + return new KeyCacheObjectImpl(num, null, -1); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementorTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementorTest.java index 31c39d3fa75f2..3a427359b9d50 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementorTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/LogicalRelImplementorTest.java @@ -126,6 +126,7 @@ public class LogicalRelImplementorTest extends GridCommonAbstractTest { NoOpMemoryTracker.INSTANCE, NoOpIoTracker.INSTANCE, 0, + null, null ) { @Override public ColocationGroup group(long srcId) { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndexTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndexTest.java index 818b1074fba8c..450e555cfb07e 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndexTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/RuntimeSortedIndexTest.java @@ -119,6 +119,7 @@ private RuntimeSortedIndex generate(RelDataType rowType, final List { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java index 8a259226bbe34..0858936705bea 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/exec/rel/AbstractExecutionTest.java @@ -307,7 +307,8 @@ protected ExecutionContext executionContext(UUID nodeId, UUID qryId, l NoOpMemoryTracker.INSTANCE, NoOpIoTracker.INSTANCE, 0, - ImmutableMap.of() + ImmutableMap.of(), + null ); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTest.java index aa0f15b1b78d1..62be28c666968 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTest.java @@ -137,7 +137,7 @@ protected QueryChecker assertQuery(String qry) { } /** */ - protected QueryChecker assertQuery(Ignite ignite, String qry) { + protected QueryChecker assertQuery(IgniteEx ignite, String qry) { return new QueryChecker(qry) { @Override protected QueryEngine getEngine() { return Commons.lookupComponent(((IgniteEx)ignite).context(), QueryEngine.class); @@ -145,12 +145,22 @@ protected QueryChecker assertQuery(Ignite ignite, String qry) { }; } - /** - * @deprecated Use {@link #sql(String, Object...)} instead. - */ + /** @deprecated Use {@link #sql(String, Object...)} instead. */ @Deprecated protected List> executeSql(String sql, Object... args) { - return sql(sql, args); + return executeSql(client, sql, args); + } + + /** @deprecated Use {@link #sql(String, Object...)} instead. */ + @Deprecated + protected List> executeSql(IgniteEx ignite, String sql, Object... args) { + CalciteQueryProcessor qryProc = Commons.lookupComponent(ignite.context(), CalciteQueryProcessor.class); + + List>> cur = qryProc.query(queryContext(), "PUBLIC", sql, args); + + try (QueryCursor> srvCursor = cur.get(0)) { + return srvCursor.getAll(); + } } /** */ @@ -171,12 +181,12 @@ protected void assertThrows(String sql, Class cls, String m /** */ protected IgniteCache createAndPopulateTable() { - return createAndPopulateTable(2, CacheMode.PARTITIONED); + return createAndPopulateTable(client, 2, CacheMode.PARTITIONED); } /** */ - protected IgniteCache createAndPopulateTable(int backups, CacheMode cacheMode) { - IgniteCache person = client.getOrCreateCache(new CacheConfiguration() + protected IgniteCache createAndPopulateTable(Ignite ignite, int backups, CacheMode cacheMode) { + IgniteCache person = ignite.getOrCreateCache(this.cacheConfiguration() .setName(TABLE_NAME) .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class) @@ -190,15 +200,25 @@ protected IgniteCache createAndPopulateTable(int backups, Cac int idx = 0; - person.put(idx++, new Employer("Igor", 10d)); - person.put(idx++, new Employer(null, 15d)); - person.put(idx++, new Employer("Ilya", 15d)); - person.put(idx++, new Employer("Roma", 10d)); - person.put(idx++, new Employer("Roma", 10d)); + put(ignite, person, idx++, new Employer("Igor", 10d)); + put(ignite, person, idx++, new Employer(null, 15d)); + put(ignite, person, idx++, new Employer("Ilya", 15d)); + put(ignite, person, idx++, new Employer("Roma", 10d)); + put(ignite, person, idx, new Employer("Roma", 10d)); return person; } + /** */ + protected void put(Ignite ignite, IgniteCache c, K key, V val) { + c.put(key, val); + } + + /** */ + protected CacheConfiguration cacheConfiguration() { + return new CacheConfiguration<>(); + } + /** */ protected CalciteQueryProcessor queryProcessor(Ignite ignite) { return Commons.lookupComponent(((IgniteEx)ignite).context(), CalciteQueryProcessor.class); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTransactionalTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTransactionalTest.java new file mode 100644 index 0000000000000..58abf9d962961 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AbstractBasicIntegrationTransactionalTest.java @@ -0,0 +1,242 @@ +/* + * 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.processors.query.calcite.integration; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; +import org.apache.ignite.internal.processors.query.QueryContext; +import org.apache.ignite.internal.processors.query.QueryEngine; +import org.apache.ignite.internal.processors.query.calcite.QueryChecker; +import org.apache.ignite.internal.processors.query.calcite.util.Commons; +import org.apache.ignite.transactions.Transaction; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; + +/** */ +@RunWith(Parameterized.class) +public abstract class AbstractBasicIntegrationTransactionalTest extends AbstractBasicIntegrationTest { + /** */ + @Parameterized.Parameter() + public SqlTransactionMode sqlTxMode; + + /** @return Test parameters. */ + @Parameterized.Parameters(name = "sqlTxMode={0}") + public static Collection parameters() { + return Arrays.asList(SqlTransactionMode.values()); + } + + /** */ + protected static SqlTransactionMode currentMode; + + /** */ + protected static Transaction tx; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration()); + cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(true); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + if (currentMode != null && sqlTxMode == currentMode) + return; + + currentMode = sqlTxMode; + + clearTransaction(); + + stopAllGrids(); + + init(); + } + + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + clearTransaction(); + + super.afterTest(); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + clearTransaction(); + + stopAllGrids(); + + currentMode = null; + + tx = null; + + super.afterTestsStopped(); + } + + /** */ + protected void init() throws Exception { + startGrids(nodeCount()); + + client = startClientGrid("client"); + } + + /** {@inheritDoc} */ + @Override protected QueryChecker assertQuery(IgniteEx ignite, String qry) { + return new QueryChecker(qry, tx, sqlTxMode) { + @Override protected QueryEngine getEngine() { + return Commons.lookupComponent(ignite.context(), QueryEngine.class); + } + }; + } + + /** {@inheritDoc} */ + @Override protected List> executeSql(IgniteEx ignite, String sql, Object... args) { + if (sqlTxMode != SqlTransactionMode.NONE && tx == null) + startTransaction(ignite); + + return super.executeSql(ignite, sql, args); + } + + /** {@inheritDoc} */ + @Override protected QueryContext queryContext() { + return QueryContext.of(tx != null ? ((TransactionProxyImpl)tx).tx().xidVersion() : null); + } + + /** {@inheritDoc} */ + @Override protected List> sql(IgniteEx ignite, String sql, Object... params) { + if (sqlTxMode != SqlTransactionMode.NONE && tx == null) + startTransaction(ignite); + + return super.sql(ignite, sql, params); + } + + /** {@inheritDoc} */ + @Override protected void put(Ignite node, IgniteCache cache, K key, V val) { + invokeAction(node, () -> { + cache.put(key, val); + return null; + }); + } + + /** {@inheritDoc} */ + @Override protected CacheConfiguration cacheConfiguration() { + return super.cacheConfiguration().setAtomicityMode(sqlTxMode == SqlTransactionMode.NONE + ? CacheAtomicityMode.ATOMIC + : CacheAtomicityMode.TRANSACTIONAL); + } + + /** */ + protected T invokeAction(Ignite node, SupplierX action) { + if (tx == null && sqlTxMode != SqlTransactionMode.NONE) + startTransaction(node); + + switch (sqlTxMode) { + case ALL: + return txAction(node, action); + case NONE: + return action.get(); + case RANDOM: + if (ThreadLocalRandom.current().nextBoolean()) + return action.get(); + else + return txAction(node, action); + default: + throw new IllegalArgumentException(); + } + } + + /** */ + public T txAction(Ignite node, SupplierX action) { + tx.resume(); + + try { + return action.get(); + } + finally { + tx.suspend(); + } + } + + /** */ + protected void startTransaction(Ignite node) { + tx = node.transactions().txStart(PESSIMISTIC, READ_COMMITTED, getTestTimeout(), 100); + + tx.suspend(); + } + + /** */ + protected void clearTransaction() { + if (tx == null) + return; + + tx.resume(); + + tx.rollback(); + + tx = null; + } + + /** */ + public String atomicity() { + return "atomicity=" + (sqlTxMode == SqlTransactionMode.NONE ? CacheAtomicityMode.ATOMIC : CacheAtomicityMode.TRANSACTIONAL); + } + + /** */ + public interface SupplierX { + /** */ + T getx() throws Exception; + + /** */ + default T get() { + try { + return getx(); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + } + + /** */ + public enum SqlTransactionMode { + /** All put, remove and SQL dml will be executed inside transaction. */ + ALL, + + /** Only some DML operations will be executed inside transaction. */ + RANDOM, + + /** Don't use transaction for DML. */ + NONE + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AggregatesIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AggregatesIntegrationTest.java index 3afa400e87342..dfe89024b4c95 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AggregatesIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/AggregatesIntegrationTest.java @@ -17,14 +17,15 @@ package org.apache.ignite.internal.processors.query.calcite.integration; +import java.util.Collections; import java.util.List; import org.apache.calcite.sql.validate.SqlConformance; import org.apache.ignite.IgniteCache; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.annotations.QuerySqlField; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.query.QueryUtils; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.testframework.GridTestUtils; @@ -33,7 +34,7 @@ /** * */ -public class AggregatesIntegrationTest extends AbstractBasicIntegrationTest { +public class AggregatesIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ @Test public void testMinMaxWithTable() { @@ -49,7 +50,7 @@ public void testMinMaxWithTable() { for (String idx : indexes) { for (int backups = -1; backups < 3; ++backups) { executeSql("create table tbl(id integer primary key, val0 integer, val1 float, val2 varchar) " + - "with template=" + (backups < 0 ? "replicated" : "partitioned,backups=" + backups)); + "with template=" + (backups < 0 ? "replicated" : "partitioned,backups=" + backups) + "," + atomicity()); executeSql("create index test_idx on tbl(" + idx + ")"); @@ -60,6 +61,8 @@ public void testMinMaxWithTable() { assertQuery("select max(val0) from tbl").returns(5).check(); assertQuery("select max(val1) from tbl").returns(50.0f).check(); + clearTransaction(); + executeSql("drop table tbl"); } } @@ -89,6 +92,8 @@ public void testMinMaxWithEntity() { assertQuery("select max(salary) from person").returns(15.0).check(); assertQuery("select max(descVal) from person").returns(15.0).check(); + clearTransaction(); + client.destroyCache(TABLE_NAME); } } @@ -101,6 +106,8 @@ public void testCountWithBackupsAndCacheModes() { assertQuery("select count(*) from person").returns(7L).check(); + clearTransaction(); + client.destroyCache(TABLE_NAME); } @@ -112,6 +119,11 @@ public void testCountWithBackupsAndCacheModes() { /** */ @Test public void testCountIndexedField() { + // Check count with two columns index. + sql("CREATE TABLE tbl (a INT, b INT, c INT) WITH " + atomicity()); + sql("CREATE INDEX idx_a ON tbl(a, c)"); + sql("CREATE INDEX idx_b ON tbl(b DESC, c)"); + createAndPopulateIndexedTable(1, CacheMode.PARTITIONED); assertQuery("select count(salary) from person").returns(4L).check(); @@ -127,11 +139,6 @@ public void testCountIndexedField() { .returns(1L, null) .check(); - // Check count with two columns index. - sql("CREATE TABLE tbl (a INT, b INT, c INT)"); - sql("CREATE INDEX idx_a ON tbl(a, c)"); - sql("CREATE INDEX idx_b ON tbl(b DESC, c)"); - for (int i = 0; i < 100; i++) { sql("INSERT INTO tbl VALUES (null, null, ?)", i % 2 == 0 ? i : null); sql("INSERT INTO tbl VALUES (?, ?, ?)", i, i, i % 2 == 0 ? null : i); @@ -149,7 +156,7 @@ public void testCountIndexedField() { */ @Test public void testGroupingByAlias() { - executeSql("CREATE TABLE t1(id INT, val_int INT, val_char VARCHAR, PRIMARY KEY(id))"); + executeSql("CREATE TABLE t1(id INT, val_int INT, val_char VARCHAR, PRIMARY KEY(id)) WITH " + atomicity()); for (int i = 0; i < 10; i++) executeSql("INSERT INTO t1 VALUES (?, ?, ?)", i, i % 3, "val" + i % 3); @@ -248,7 +255,7 @@ public void testCountOfNonNumericField() { /** */ @SuppressWarnings("ThrowableNotThrown") @Test - public void testMultipleRowsFromSingleAggr() throws IgniteCheckedException { + public void testMultipleRowsFromSingleAggr() { createAndPopulateTable(); GridTestUtils.assertThrowsWithCause(() -> assertQuery("SELECT (SELECT name FROM person)").check(), @@ -267,10 +274,12 @@ public void testMultipleRowsFromSingleAggr() throws IgniteCheckedException { IgniteCache person = client.cache(cacheName); + clearTransaction(); + person.clear(); for (int gridIdx = 0; gridIdx < nodeCount(); gridIdx++) - person.put(primaryKey(grid(gridIdx).cache(cacheName)), new Employer(gridIdx == 0 ? "Emp" : null, 0.0d)); + put(client, person, primaryKey(grid(gridIdx).cache(cacheName)), new Employer(gridIdx == 0 ? "Emp" : null, 0.0d)); GridTestUtils.assertThrowsWithCause(() -> assertQuery("SELECT (SELECT name FROM person)").check(), IllegalArgumentException.class); @@ -307,12 +316,12 @@ public void testAnyValAggr() { /** */ @Test - public void testColocatedAggregate() throws Exception { + public void testColocatedAggregate() { executeSql("CREATE TABLE t1(id INT, val0 VARCHAR, val1 VARCHAR, val2 VARCHAR, PRIMARY KEY(id, val1)) " + - "WITH AFFINITY_KEY=val1"); + "WITH AFFINITY_KEY=val1," + atomicity()); executeSql("CREATE TABLE t2(id INT, val0 VARCHAR, val1 VARCHAR, val2 VARCHAR, PRIMARY KEY(id, val1)) " + - "WITH AFFINITY_KEY=val1"); + "WITH AFFINITY_KEY=val1," + atomicity()); for (int i = 0; i < 100; i++) executeSql("INSERT INTO t1 VALUES (?, ?, ?, ?)", i, "val" + i, "val" + i % 2, "val" + i); @@ -339,8 +348,8 @@ public void testColocatedAggregate() throws Exception { /** */ @Test - public void testEverySomeAggregate() throws Exception { - executeSql("CREATE TABLE t(c1 INT, c2 INT)"); + public void testEverySomeAggregate() { + executeSql("CREATE TABLE t(c1 INT, c2 INT) WITH " + atomicity()); executeSql("INSERT INTO t VALUES (null, 0)"); executeSql("INSERT INTO t VALUES (0, null)"); executeSql("INSERT INTO t VALUES (null, null)"); @@ -355,9 +364,32 @@ public void testEverySomeAggregate() throws Exception { assertQuery("SELECT SOME(c1 > c2) FROM t").returns(false).check(); } + /** */ + @Test + public void testCountIndexedFieldSegmented() { + client.getOrCreateCache(cacheConfiguration() + .setName("cache_seg") + .setSqlSchema(QueryUtils.DFLT_SCHEMA) + .setQueryEntities(Collections.singleton( + new QueryEntity() + .setKeyType(Integer.class.getName()) + .setValueType("tbl_seg") + .setTableName("TBL_SEG") + .addQueryField("a", Integer.class.getName(), null) + .addQueryField("b", Integer.class.getName(), null) + .setIndexes(F.asList(new QueryIndex("a", true), new QueryIndex("b", false))))) + .setQueryParallelism(5)); + + for (int i = 0; i < 100; i++) + sql("INSERT INTO tbl_seg (_key, a, b) VALUES (?, ?, ?)", i, i % 2 == 0 ? i : null, i % 2 == 0 ? null : i); + + assertQuery("SELECT COUNT(a) FROM tbl_seg").returns(50L).check(); + assertQuery("SELECT COUNT(b) FROM tbl_seg").returns(50L).check(); + } + /** */ protected void createAndPopulateIndexedTable(int backups, CacheMode cacheMode) { - IgniteCache person = client.getOrCreateCache(new CacheConfiguration() + IgniteCache person = client.getOrCreateCache(this.cacheConfiguration() .setName(TABLE_NAME) .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, IndexedEmployer.class).setTableName(TABLE_NAME))) @@ -367,13 +399,13 @@ protected void createAndPopulateIndexedTable(int backups, CacheMode cacheMode) { int idx = 0; - person.put(idx++, new IndexedEmployer("Igor", 5d, 9d)); - person.put(idx++, new IndexedEmployer(null, 3d, null)); - person.put(idx++, new IndexedEmployer("Ilya", 1d, 1d)); - person.put(idx++, new IndexedEmployer("Roma", null, 9d)); - person.put(idx++, new IndexedEmployer(null, null, null)); - person.put(idx++, new IndexedEmployer("Oleg", 15d, 15d)); - person.put(idx++, new IndexedEmployer("Maya", null, null)); + put(client, person, idx++, new IndexedEmployer("Igor", 5d, 9d)); + put(client, person, idx++, new IndexedEmployer(null, 3d, null)); + put(client, person, idx++, new IndexedEmployer("Ilya", 1d, 1d)); + put(client, person, idx++, new IndexedEmployer("Roma", null, 9d)); + put(client, person, idx++, new IndexedEmployer(null, null, null)); + put(client, person, idx++, new IndexedEmployer("Oleg", 15d, 15d)); + put(client, person, idx, new IndexedEmployer("Maya", null, null)); } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java index 43f6fbb496ca1..281463152981a 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CalciteBasicSecondaryIndexIntegrationTest.java @@ -46,7 +46,7 @@ /** * Basic index tests. */ -public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicIntegrationTest { +public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ private static final String PK_IDX_NAME = QueryUtils.PRIMARY_KEY_INDEX; @@ -69,8 +69,8 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte private static final String NAME_DATE_IDX = "NAME_DATE_IDX"; /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); + @Override protected void init() throws Exception { + super.init(); QueryEntity projEntity = new QueryEntity(); projEntity.setKeyType(Integer.class.getName()); @@ -105,35 +105,6 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte IgniteCache devCache = client.createCache(projCfg); - devCache.put(1, new Developer("Mozart", 3, "Vienna", 33)); - devCache.put(2, new Developer("Beethoven", 2, "Vienna", 44)); - devCache.put(3, new Developer("Bach", 1, "Leipzig", 55)); - devCache.put(4, new Developer("Strauss", 2, "Munich", 66)); - - devCache.put(5, new Developer("Vagner", 4, "Leipzig", 70)); - devCache.put(6, new Developer("Chaikovsky", 5, "Votkinsk", 53)); - devCache.put(7, new Developer("Verdy", 6, "Rankola", 88)); - devCache.put(8, new Developer("Stravinsky", 7, "Spt", 89)); - devCache.put(9, new Developer("Rahmaninov", 8, "Starorussky ud", 70)); - devCache.put(10, new Developer("Shubert", 9, "Vienna", 31)); - devCache.put(11, new Developer("Glinka", 10, "Smolenskaya gb", 53)); - - devCache.put(12, new Developer("Einaudi", 11, "", -1)); - devCache.put(13, new Developer("Glass", 12, "", -1)); - devCache.put(14, new Developer("Rihter", 13, "", -1)); - - devCache.put(15, new Developer("Marradi", 14, "", -1)); - devCache.put(16, new Developer("Zimmer", 15, "", -1)); - devCache.put(17, new Developer("Hasaishi", 16, "", -1)); - - devCache.put(18, new Developer("Arnalds", 17, "", -1)); - devCache.put(19, new Developer("Yiruma", 18, "", -1)); - devCache.put(20, new Developer("O'Halloran", 19, "", -1)); - - devCache.put(21, new Developer("Cacciapaglia", 20, "", -1)); - devCache.put(22, new Developer("Prokofiev", 21, "", -1)); - devCache.put(23, new Developer("Musorgskii", 22, "", -1)); - QueryEntity bdEntity = new QueryEntity(); bdEntity.setKeyType(Integer.class.getName()); bdEntity.setKeyFieldName("id"); @@ -158,16 +129,8 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte IgniteCache bdCache = client.createCache(bdCfg); - bdCache.put(1, new Birthday("Mozart", Date.valueOf("1756-01-27"))); - bdCache.put(2, new Birthday("Beethoven", null)); - bdCache.put(3, new Birthday("Bach", Date.valueOf("1685-03-31"))); - bdCache.put(4, new Birthday("Strauss", Date.valueOf("1864-06-11"))); - bdCache.put(5, new Birthday("Vagner", Date.valueOf("1813-05-22"))); - bdCache.put(6, new Birthday("Chaikovsky", Date.valueOf("1840-05-07"))); - bdCache.put(7, new Birthday("Verdy", Date.valueOf("1813-10-10"))); - IgniteCache tblWithAff = - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("TBL_WITH_AFF_KEY") .setSqlSchema("PUBLIC") .setBackups(1) @@ -175,11 +138,8 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte .setTableName("TBL_WITH_AFF_KEY"))) ); - tblWithAff.put(new CalciteQueryProcessorTest.Key(1, 2), new CalciteQueryProcessorTest.Developer("Petr", 10)); - tblWithAff.put(new CalciteQueryProcessorTest.Key(2, 3), new CalciteQueryProcessorTest.Developer("Ivan", 11)); - IgniteCache tblConstrPk = - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("TBL_CONSTR_PK") .setSqlSchema("PUBLIC") .setBackups(0) @@ -189,11 +149,51 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte .addQueryField("id", Integer.class.getName(), null))) ); - tblConstrPk.put(1, new CalciteQueryProcessorTest.Developer("Petr", 10)); - tblConstrPk.put(2, new CalciteQueryProcessorTest.Developer("Ivan", 11)); - executeSql("CREATE TABLE PUBLIC.UNWRAP_PK" + " (F1 VARCHAR, F2 BIGINT, F3 BIGINT, F4 BIGINT, " + - "CONSTRAINT PK PRIMARY KEY (F2, F1)) WITH \"backups=0, affinity_key=F1\""); + "CONSTRAINT PK PRIMARY KEY (F2, F1)) WITH \"backups=0, affinity_key=F1," + atomicity() + "\""); + + put(client, devCache, 1, new Developer("Mozart", 3, "Vienna", 33)); + put(client, devCache, 2, new Developer("Beethoven", 2, "Vienna", 44)); + put(client, devCache, 3, new Developer("Bach", 1, "Leipzig", 55)); + put(client, devCache, 4, new Developer("Strauss", 2, "Munich", 66)); + + put(client, devCache, 5, new Developer("Vagner", 4, "Leipzig", 70)); + put(client, devCache, 6, new Developer("Chaikovsky", 5, "Votkinsk", 53)); + put(client, devCache, 7, new Developer("Verdy", 6, "Rankola", 88)); + put(client, devCache, 8, new Developer("Stravinsky", 7, "Spt", 89)); + put(client, devCache, 9, new Developer("Rahmaninov", 8, "Starorussky ud", 70)); + put(client, devCache, 10, new Developer("Shubert", 9, "Vienna", 31)); + put(client, devCache, 11, new Developer("Glinka", 10, "Smolenskaya gb", 53)); + + put(client, devCache, 12, new Developer("Einaudi", 11, "", -1)); + put(client, devCache, 13, new Developer("Glass", 12, "", -1)); + put(client, devCache, 14, new Developer("Rihter", 13, "", -1)); + + put(client, devCache, 15, new Developer("Marradi", 14, "", -1)); + put(client, devCache, 16, new Developer("Zimmer", 15, "", -1)); + put(client, devCache, 17, new Developer("Hasaishi", 16, "", -1)); + + put(client, devCache, 18, new Developer("Arnalds", 17, "", -1)); + put(client, devCache, 19, new Developer("Yiruma", 18, "", -1)); + put(client, devCache, 20, new Developer("O'Halloran", 19, "", -1)); + + put(client, devCache, 21, new Developer("Cacciapaglia", 20, "", -1)); + put(client, devCache, 22, new Developer("Prokofiev", 21, "", -1)); + put(client, devCache, 23, new Developer("Musorgskii", 22, "", -1)); + + put(client, bdCache, 1, new Birthday("Mozart", Date.valueOf("1756-01-27"))); + put(client, bdCache, 2, new Birthday("Beethoven", null)); + put(client, bdCache, 3, new Birthday("Bach", Date.valueOf("1685-03-31"))); + put(client, bdCache, 4, new Birthday("Strauss", Date.valueOf("1864-06-11"))); + put(client, bdCache, 5, new Birthday("Vagner", Date.valueOf("1813-05-22"))); + put(client, bdCache, 6, new Birthday("Chaikovsky", Date.valueOf("1840-05-07"))); + put(client, bdCache, 7, new Birthday("Verdy", Date.valueOf("1813-10-10"))); + + put(client, tblWithAff, new CalciteQueryProcessorTest.Key(1, 2), new CalciteQueryProcessorTest.Developer("Petr", 10)); + put(client, tblWithAff, new CalciteQueryProcessorTest.Key(2, 3), new CalciteQueryProcessorTest.Developer("Ivan", 11)); + + put(client, tblConstrPk, 1, new CalciteQueryProcessorTest.Developer("Petr", 10)); + put(client, tblConstrPk, 2, new CalciteQueryProcessorTest.Developer("Ivan", 11)); executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Petr', 1, 2, 3)"); executeSql("INSERT INTO PUBLIC.UNWRAP_PK(F1, F2, F3, F4) values ('Ivan', 2, 2, 4)"); @@ -219,7 +219,7 @@ public class CalciteBasicSecondaryIndexIntegrationTest extends AbstractBasicInte /** */ private CacheConfiguration cache(QueryEntity ent) { - return new CacheConfiguration(ent.getTableName()) + return this.cacheConfiguration().setName(ent.getTableName()) .setCacheMode(CacheMode.PARTITIONED) .setBackups(1) .setQueryEntities(singletonList(ent)) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CorrelatesIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CorrelatesIntegrationTest.java index b1d8fc330b40f..92784fe29a2ca 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CorrelatesIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/CorrelatesIntegrationTest.java @@ -23,13 +23,13 @@ /** * Tests correlated queries. */ -public class CorrelatesIntegrationTest extends AbstractBasicIntegrationTest { +public class CorrelatesIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** * Checks correlates are assigned before access. */ @Test public void testCorrelatesAssignedBeforeAccess() { - sql("create table test_tbl(v INTEGER)"); + sql("create table test_tbl(v INTEGER) WITH " + atomicity()); sql("INSERT INTO test_tbl VALUES (1)"); assertQuery("SELECT t0.v, (SELECT t0.v + t1.v FROM test_tbl t1) AS j FROM test_tbl t0") @@ -44,8 +44,8 @@ public void testCorrelatesAssignedBeforeAccess() { public void testCorrelatesDifferentDataType() { for (String type : new String[] {"INTEGER", "TINYINT"}) { try { - sql("CREATE TABLE t1(v INTEGER)"); - sql("CREATE TABLE t2(v " + type + ")"); + sql("CREATE TABLE t1(v INTEGER) WITH " + atomicity()); + sql("CREATE TABLE t2(v " + type + ") WITH " + atomicity()); sql("INSERT INTO t1 VALUES (1)"); sql("INSERT INTO t2 VALUES (1)"); @@ -54,6 +54,8 @@ public void testCorrelatesDifferentDataType() { .check(); } finally { + clearTransaction(); + sql("DROP TABLE t1"); sql("DROP TABLE t2"); } @@ -65,7 +67,7 @@ public void testCorrelatesDifferentDataType() { */ @Test public void testCorrelatesWithTableSpool() { - sql("CREATE TABLE test(i1 INT, i2 INT)"); + sql("CREATE TABLE test(i1 INT, i2 INT) WITH " + atomicity()); sql("INSERT INTO test VALUES (1, 1), (2, 2)"); assertQuery("SELECT (SELECT t1.i1 + t1.i2 + t0.i2 FROM test t1 WHERE i1 = 1) FROM test t0") @@ -80,9 +82,9 @@ public void testCorrelatesWithTableSpool() { */ @Test public void testCorrelatesCollision() { - sql("CREATE TABLE test1 (a INTEGER, b INTEGER)"); + sql("CREATE TABLE test1 (a INTEGER, b INTEGER) WITH " + atomicity()); + sql("CREATE TABLE test2 (a INTEGER, c INTEGER) WITH " + atomicity()); sql("INSERT INTO test1 VALUES (11, 1), (12, 2), (13, 3)"); - sql("CREATE TABLE test2 (a INTEGER, c INTEGER)"); sql("INSERT INTO test2 VALUES (11, 1), (12, 1), (13, 4)"); // Collision by correlate variables in the left hand. @@ -105,9 +107,9 @@ public void testCorrelatesCollision() { */ @Test public void testCorrelatedDistribution() { - sql("CREATE TABLE dept(deptid INTEGER, name VARCHAR, PRIMARY KEY(deptid))"); + sql("CREATE TABLE dept(deptid INTEGER, name VARCHAR, PRIMARY KEY(deptid)) WITH " + atomicity()); sql("CREATE TABLE emp(empid INTEGER, deptid INTEGER, name VARCHAR, PRIMARY KEY(empid, deptid)) " + - "WITH AFFINITY_KEY=deptid"); + "WITH AFFINITY_KEY=deptid," + atomicity()); sql("INSERT INTO dept VALUES (0, 'dept0'), (1, 'dept1'), (2, 'dept2')"); sql("INSERT INTO emp VALUES (0, 0, 'emp0'), (1, 0, 'emp1'), (2, 0, 'emp2'), " + diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DataTypesTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DataTypesTest.java index 7a35de837caf6..d0a641afd7eb5 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DataTypesTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DataTypesTest.java @@ -31,80 +31,75 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteException; import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.calcite.hint.HintDefinition; import org.apache.ignite.internal.util.typedef.F; import org.junit.Test; import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.FRAMEWORK_CONFIG; +import static org.junit.Assume.assumeTrue; /** * Test SQL data types. */ -public class DataTypesTest extends AbstractBasicIntegrationTest { +public class DataTypesTest extends AbstractBasicIntegrationTransactionalTest { /** Tests Other type. */ @Test public void testOtherType() { - try { - executeSql("CREATE TABLE t(id INT, oth OTHER)"); - - assertThrows("CREATE TABLE t2(id INT, oth OTHER DEFAULT 'str')", IgniteSQLException.class, - "Type 'OTHER' doesn't support default value."); - - executeSql("INSERT INTO t VALUES (1, 'str')"); - executeSql("INSERT INTO t VALUES (2, 22)"); - executeSql("INSERT INTO t VALUES (3, CAST('33.5' AS FLOAT))"); - executeSql("INSERT INTO t VALUES (4, CAST('44.5' AS DOUBLE))"); - executeSql("INSERT INTO t VALUES (5, CAST('fd10556e-fc27-4a99-b5e4-89b8344cb3ce' as UUID))"); - executeSql("INSERT INTO t VALUES (6, NULL)"); - executeSql("INSERT INTO t VALUES (7, NULL)"); - - assertQuery("SELECT oth FROM t order by id") - .ordered() - .returns("str") - .returns(22) - .returns(33.5f) - .returns(44.5d) - .returns(UUID.fromString("fd10556e-fc27-4a99-b5e4-89b8344cb3ce")) - .returns(new Object[] {null}) - .returns(new Object[] {null}) - .check(); - - assertThrows("SELECT MIN(oth) FROM t", UnsupportedOperationException.class, - "MIN() is not supported for type 'OTHER'."); - - assertThrows("SELECT MAX(oth) FROM t", UnsupportedOperationException.class, - "MAX() is not supported for type 'OTHER'."); - - assertThrows("SELECT AVG(oth) from t", UnsupportedOperationException.class, - "AVG() is not supported for type 'OTHER'."); - - assertThrows("SELECT oth from t WHERE oth > 0", CalciteException.class, - "Invalid types for comparison"); - - assertQuery("SELECT oth FROM t WHERE oth=22") - .returns(22) - .check(); - - assertQuery("SELECT oth FROM t WHERE oth is NULL") - .returns(new Object[] {null}) - .returns(new Object[] {null}) - .check(); - - executeSql("DELETE FROM t WHERE id IN (2, 3, 4, 5)"); - - assertQuery("SELECT oth FROM t WHERE oth is not NULL") - .returns("str") - .check(); - - assertQuery("SELECT oth FROM t WHERE oth!=22") - .returns("str") - .check(); - } - finally { - executeSql("DROP TABLE IF EXISTS t"); - } + executeSql("CREATE TABLE t(id INT, oth OTHER) WITH " + atomicity()); + + assertThrows("CREATE TABLE t2(id INT, oth OTHER DEFAULT 'str')", IgniteSQLException.class, + "Type 'OTHER' doesn't support default value."); + + executeSql("INSERT INTO t VALUES (1, 'str')"); + executeSql("INSERT INTO t VALUES (2, 22)"); + executeSql("INSERT INTO t VALUES (3, CAST('33.5' AS FLOAT))"); + executeSql("INSERT INTO t VALUES (4, CAST('44.5' AS DOUBLE))"); + executeSql("INSERT INTO t VALUES (5, CAST('fd10556e-fc27-4a99-b5e4-89b8344cb3ce' as UUID))"); + executeSql("INSERT INTO t VALUES (6, NULL)"); + executeSql("INSERT INTO t VALUES (7, NULL)"); + + assertQuery("SELECT oth FROM t order by id") + .ordered() + .returns("str") + .returns(22) + .returns(33.5f) + .returns(44.5d) + .returns(UUID.fromString("fd10556e-fc27-4a99-b5e4-89b8344cb3ce")) + .returns(new Object[]{null}) + .returns(new Object[]{null}) + .check(); + + assertThrows("SELECT MIN(oth) FROM t", UnsupportedOperationException.class, + "MIN() is not supported for type 'OTHER'."); + + assertThrows("SELECT MAX(oth) FROM t", UnsupportedOperationException.class, + "MAX() is not supported for type 'OTHER'."); + + assertThrows("SELECT AVG(oth) from t", UnsupportedOperationException.class, + "AVG() is not supported for type 'OTHER'."); + + assertThrows("SELECT oth from t WHERE oth > 0", CalciteException.class, + "Invalid types for comparison"); + + assertQuery("SELECT oth FROM t WHERE oth=22") + .returns(22) + .check(); + + assertQuery("SELECT oth FROM t WHERE oth is NULL") + .returns(new Object[]{null}) + .returns(new Object[]{null}) + .check(); + + executeSql("DELETE FROM t WHERE id IN (2, 3, 4, 5)"); + + assertQuery("SELECT oth FROM t WHERE oth is not NULL") + .returns("str") + .check(); + + assertQuery("SELECT oth FROM t WHERE oth!=22") + .returns("str") + .check(); } /** Tests UUID without index. */ @@ -121,92 +116,87 @@ public void testUuidWithIndex() { /** Tests UUID type. */ private void testUuid(boolean indexed) { - try { - executeSql("CREATE TABLE t(id INT, name VARCHAR(255), uid UUID, primary key (id))"); - - if (indexed) - executeSql("CREATE INDEX uuid_idx ON t (uid);"); - - UUID uuid1 = UUID.randomUUID(); - UUID uuid2 = UUID.randomUUID(); - - UUID max = uuid1.compareTo(uuid2) > 0 ? uuid1 : uuid2; - UUID min = max == uuid1 ? uuid2 : uuid1; - - executeSql("INSERT INTO t VALUES (1, 'fd10556e-fc27-4a99-b5e4-89b8344cb3ce', '" + uuid1 + "')"); - // Name == UUID - executeSql("INSERT INTO t VALUES (2, '" + uuid2 + "', '" + uuid2 + "')"); - executeSql("INSERT INTO t VALUES (3, NULL, NULL)"); - - assertQuery("SELECT * FROM t") - .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) - .returns(2, uuid2.toString(), uuid2) - .returns(3, null, null) - .check(); - - assertQuery("SELECT uid FROM t WHERE uid < '" + max + "'") - .returns(min) - .check(); - - assertQuery("SELECT uid FROM t WHERE '" + max + "' > uid") - .returns(min) - .check(); - - assertQuery("SELECT * FROM t WHERE name = uid") - .returns(2, uuid2.toString(), uuid2) - .check(); - - assertQuery("SELECT * FROM t WHERE name != uid") - .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) - .check(); - - assertQuery("SELECT count(*), uid FROM t group by uid order by uid") - .returns(1L, min) - .returns(1L, max) - .returns(1L, null) - .check(); - - assertQuery("SELECT count(*), uid FROM t group by uid having uid = " + - "'" + uuid1 + "' order by uid") - .returns(1L, uuid1) - .check(); - - assertQuery("SELECT t1.* from t t1, (select * from t) t2 where t1.uid = t2.name") - .returns(2, uuid2.toString(), uuid2) - .check(); - - assertQuery("SELECT t1.* from t t1, (select * from t) t2 " + - "where t1.uid = t2.uid") - .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) - .returns(2, uuid2.toString(), uuid2) - .check(); - - assertQuery("SELECT max(uid) from t") - .returns(max) - .check(); - - assertQuery("SELECT min(uid) from t") - .returns(min) - .check(); - - assertQuery("SELECT uid from t where uid is not NULL order by uid") - .returns(min) - .returns(max) - .check(); - - assertQuery("SELECT uid from t where uid is NULL order by uid") - .returns(new Object[] {null}) - .check(); - - assertThrows("SELECT avg(uid) from t", UnsupportedOperationException.class, - "AVG() is not supported for type 'UUID'."); - - assertThrows("SELECT sum(uid) from t", UnsupportedOperationException.class, - "SUM() is not supported for type 'UUID'."); - } - finally { - executeSql("DROP TABLE if exists tbl"); - } + executeSql("CREATE TABLE t(id INT, name VARCHAR(255), uid UUID, primary key (id)) WITH " + atomicity()); + + if (indexed) + executeSql("CREATE INDEX uuid_idx ON t (uid);"); + + UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); + + UUID max = uuid1.compareTo(uuid2) > 0 ? uuid1 : uuid2; + UUID min = max == uuid1 ? uuid2 : uuid1; + + executeSql("INSERT INTO t VALUES (1, 'fd10556e-fc27-4a99-b5e4-89b8344cb3ce', '" + uuid1 + "')"); + // Name == UUID + executeSql("INSERT INTO t VALUES (2, '" + uuid2 + "', '" + uuid2 + "')"); + executeSql("INSERT INTO t VALUES (3, NULL, NULL)"); + + assertQuery("SELECT * FROM t") + .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) + .returns(2, uuid2.toString(), uuid2) + .returns(3, null, null) + .check(); + + assertQuery("SELECT uid FROM t WHERE uid < '" + max + "'") + .returns(min) + .check(); + + assertQuery("SELECT uid FROM t WHERE '" + max + "' > uid") + .returns(min) + .check(); + + assertQuery("SELECT * FROM t WHERE name = uid") + .returns(2, uuid2.toString(), uuid2) + .check(); + + assertQuery("SELECT * FROM t WHERE name != uid") + .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) + .check(); + + assertQuery("SELECT count(*), uid FROM t group by uid order by uid") + .returns(1L, min) + .returns(1L, max) + .returns(1L, null) + .check(); + + assertQuery("SELECT count(*), uid FROM t group by uid having uid = " + + "'" + uuid1 + "' order by uid") + .returns(1L, uuid1) + .check(); + + assertQuery("SELECT t1.* from t t1, (select * from t) t2 where t1.uid = t2.name") + .returns(2, uuid2.toString(), uuid2) + .check(); + + assertQuery("SELECT t1.* from t t1, (select * from t) t2 " + + "where t1.uid = t2.uid") + .returns(1, "fd10556e-fc27-4a99-b5e4-89b8344cb3ce", uuid1) + .returns(2, uuid2.toString(), uuid2) + .check(); + + assertQuery("SELECT max(uid) from t") + .returns(max) + .check(); + + assertQuery("SELECT min(uid) from t") + .returns(min) + .check(); + + assertQuery("SELECT uid from t where uid is not NULL order by uid") + .returns(min) + .returns(max) + .check(); + + assertQuery("SELECT uid from t where uid is NULL order by uid") + .returns(new Object[]{null}) + .check(); + + assertThrows("SELECT avg(uid) from t", UnsupportedOperationException.class, + "AVG() is not supported for type 'UUID'."); + + assertThrows("SELECT sum(uid) from t", UnsupportedOperationException.class, + "SUM() is not supported for type 'UUID'."); } /** @@ -214,30 +204,25 @@ private void testUuid(boolean indexed) { */ @Test public void testNumericRanges() { - try { - executeSql("CREATE TABLE tbl(tiny TINYINT, small SMALLINT, i INTEGER, big BIGINT)"); + executeSql("CREATE TABLE tbl(tiny TINYINT, small SMALLINT, i INTEGER, big BIGINT) WITH " + atomicity()); - executeSql("INSERT INTO tbl VALUES (" + Byte.MAX_VALUE + ", " + Short.MAX_VALUE + ", " + - Integer.MAX_VALUE + ", " + Long.MAX_VALUE + ')'); + executeSql("INSERT INTO tbl VALUES (" + Byte.MAX_VALUE + ", " + Short.MAX_VALUE + ", " + + Integer.MAX_VALUE + ", " + Long.MAX_VALUE + ')'); - assertQuery("SELECT tiny FROM tbl").returns(Byte.MAX_VALUE).check(); - assertQuery("SELECT small FROM tbl").returns(Short.MAX_VALUE).check(); - assertQuery("SELECT i FROM tbl").returns(Integer.MAX_VALUE).check(); - assertQuery("SELECT big FROM tbl").returns(Long.MAX_VALUE).check(); + assertQuery("SELECT tiny FROM tbl").returns(Byte.MAX_VALUE).check(); + assertQuery("SELECT small FROM tbl").returns(Short.MAX_VALUE).check(); + assertQuery("SELECT i FROM tbl").returns(Integer.MAX_VALUE).check(); + assertQuery("SELECT big FROM tbl").returns(Long.MAX_VALUE).check(); - executeSql("DELETE from tbl"); + executeSql("DELETE from tbl"); - executeSql("INSERT INTO tbl VALUES (" + Byte.MIN_VALUE + ", " + Short.MIN_VALUE + ", " + - Integer.MIN_VALUE + ", " + Long.MIN_VALUE + ')'); + executeSql("INSERT INTO tbl VALUES (" + Byte.MIN_VALUE + ", " + Short.MIN_VALUE + ", " + + Integer.MIN_VALUE + ", " + Long.MIN_VALUE + ')'); - assertQuery("SELECT tiny FROM tbl").returns(Byte.MIN_VALUE).check(); - assertQuery("SELECT small FROM tbl").returns(Short.MIN_VALUE).check(); - assertQuery("SELECT i FROM tbl").returns(Integer.MIN_VALUE).check(); - assertQuery("SELECT big FROM tbl").returns(Long.MIN_VALUE).check(); - } - finally { - executeSql("DROP TABLE if exists tbl"); - } + assertQuery("SELECT tiny FROM tbl").returns(Byte.MIN_VALUE).check(); + assertQuery("SELECT small FROM tbl").returns(Short.MIN_VALUE).check(); + assertQuery("SELECT i FROM tbl").returns(Integer.MIN_VALUE).check(); + assertQuery("SELECT big FROM tbl").returns(Long.MIN_VALUE).check(); } /** @@ -245,38 +230,33 @@ public void testNumericRanges() { */ @Test public void testNumericConvertationOnEquals() { - try { - executeSql("CREATE TABLE tbl(tiny TINYINT, small SMALLINT, i INTEGER, big BIGINT)"); + executeSql("CREATE TABLE tbl(tiny TINYINT, small SMALLINT, i INTEGER, big BIGINT) WITH " + atomicity()); - executeSql("INSERT INTO tbl VALUES (1, 2, 3, 4), (5, 5, 5, 5)"); + executeSql("INSERT INTO tbl VALUES (1, 2, 3, 4), (5, 5, 5, 5)"); - assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.small)").returns((byte)5).check(); - assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.tiny)").returns((short)5).check(); + assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.small)").returns((byte)5).check(); + assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.tiny)").returns((short)5).check(); - assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.i)").returns((byte)5).check(); - assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.tiny)").returns(5).check(); + assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.i)").returns((byte)5).check(); + assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.tiny)").returns(5).check(); - assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.big)").returns((byte)5).check(); - assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.tiny)").returns(5L).check(); + assertQuery("SELECT t1.tiny FROM tbl t1 JOIN tbl t2 ON (t1.tiny=t2.big)").returns((byte)5).check(); + assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.tiny)").returns(5L).check(); - assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.i)").returns((short)5).check(); - assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.small)").returns(5).check(); + assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.i)").returns((short)5).check(); + assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.small)").returns(5).check(); - assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.big)").returns((short)5).check(); - assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.small)").returns(5L).check(); + assertQuery("SELECT t1.small FROM tbl t1 JOIN tbl t2 ON (t1.small=t2.big)").returns((short)5).check(); + assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.small)").returns(5L).check(); - assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.big)").returns(5).check(); - assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.i)").returns(5L).check(); - } - finally { - executeSql("DROP TABLE if exists tbl"); - } + assertQuery("SELECT t1.i FROM tbl t1 JOIN tbl t2 ON (t1.i=t2.big)").returns(5).check(); + assertQuery("SELECT t1.big FROM tbl t1 JOIN tbl t2 ON (t1.big=t2.i)").returns(5L).check(); } /** */ @Test public void testUnicodeStrings() { - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("string_cache") .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, String.class).setTableName("string_table"))) @@ -320,36 +300,31 @@ public void testUnicodeStrings() { /** */ @Test public void testBinarySql() { - try { - executeSql("CREATE TABLE tbl(b BINARY(3), v VARBINARY)"); + executeSql("CREATE TABLE tbl(b BINARY(3), v VARBINARY) WITH " + atomicity()); - byte[] val = new byte[] {1, 2, 3}; + byte[] val = new byte[]{1, 2, 3}; - // From parameters to internal, from internal to store, from store to internal and from internal to user. - executeSql("INSERT INTO tbl VALUES (?, ?)", val, val); + // From parameters to internal, from internal to store, from store to internal and from internal to user. + executeSql("INSERT INTO tbl VALUES (?, ?)", val, val); - List> res = executeSql("SELECT b, v FROM tbl"); + List> res = executeSql("SELECT b, v FROM tbl"); - assertEquals(1, res.size()); - assertEquals(2, res.get(0).size()); - assertTrue(Objects.deepEquals(val, res.get(0).get(0))); - assertTrue(Objects.deepEquals(val, res.get(0).get(1))); + assertEquals(1, res.size()); + assertEquals(2, res.get(0).size()); + assertTrue(Objects.deepEquals(val, res.get(0).get(0))); + assertTrue(Objects.deepEquals(val, res.get(0).get(1))); - executeSql("DELETE FROM tbl"); + executeSql("DELETE FROM tbl"); - // From literal to internal, from internal to store, from store to internal and from internal to user. - executeSql("INSERT INTO tbl VALUES (x'1A2B3C', x'AABBCC')"); + // From literal to internal, from internal to store, from store to internal and from internal to user. + executeSql("INSERT INTO tbl VALUES (x'1A2B3C', x'AABBCC')"); - res = executeSql("SELECT b, v FROM tbl"); + res = executeSql("SELECT b, v FROM tbl"); - assertEquals(1, res.size()); - assertEquals(2, res.get(0).size()); - assertTrue(Objects.deepEquals(new byte[] {0x1A, 0x2B, 0x3C}, res.get(0).get(0))); - assertTrue(Objects.deepEquals(new byte[] {(byte)0xAA, (byte)0xBB, (byte)0xCC}, res.get(0).get(1))); - } - finally { - executeSql("DROP TABLE IF EXISTS tbl"); - } + assertEquals(1, res.size()); + assertEquals(2, res.get(0).size()); + assertTrue(Objects.deepEquals(new byte[]{0x1A, 0x2B, 0x3C}, res.get(0).get(0))); + assertTrue(Objects.deepEquals(new byte[]{(byte)0xAA, (byte)0xBB, (byte)0xCC}, res.get(0).get(1))); } /** */ @@ -377,74 +352,70 @@ public void testUnsupportedTypes() { /** Cache API - SQL API cross check. */ @Test public void testBinaryCache() { - try { - IgniteCache cache = client.getOrCreateCache(new CacheConfiguration() - .setName("binary_cache") - .setSqlSchema("PUBLIC") - .setQueryEntities(F.asList(new QueryEntity(Integer.class, byte[].class).setTableName("binary_table"))) - ); + IgniteCache cache = client.getOrCreateCache(this.cacheConfiguration() + .setName("binary_cache") + .setSqlSchema("PUBLIC") + .setQueryEntities(F.asList(new QueryEntity(Integer.class, byte[].class).setTableName("binary_table"))) + ); + + byte[] val = new byte[]{1, 2, 3}; - byte[] val = new byte[] {1, 2, 3}; + // From cache to internal, from internal to user. + put(client, cache, 1, val); - // From cache to internal, from internal to user. - cache.put(1, val); + List> res = executeSql("SELECT _val FROM binary_table"); + assertEquals(1, res.size()); + assertEquals(1, res.get(0).size()); + assertTrue(Objects.deepEquals(val, res.get(0).get(0))); - List> res = executeSql("SELECT _val FROM binary_table"); - assertEquals(1, res.size()); - assertEquals(1, res.get(0).size()); - assertTrue(Objects.deepEquals(val, res.get(0).get(0))); + // From literal to internal, from internal to cache. + executeSql("INSERT INTO binary_table (_KEY, _VAL) VALUES (2, x'010203')"); - // From literal to internal, from internal to cache. - executeSql("INSERT INTO binary_table (_KEY, _VAL) VALUES (2, x'010203')"); + SupplierX check = () -> { byte[] resVal = cache.get(2); + assertTrue(Objects.deepEquals(val, resVal)); - } - finally { - client.destroyCache("binary_cache"); - } + + return null; + }; + + if (sqlTxMode == SqlTransactionMode.NONE) + check.get(); + else + txAction(client, check); } /** */ @Test public void testBinaryAggregation() { - try { - executeSql("CREATE TABLE tbl(b varbinary)"); - executeSql("INSERT INTO tbl VALUES (NULL)"); - executeSql("INSERT INTO tbl VALUES (x'010203')"); - executeSql("INSERT INTO tbl VALUES (x'040506')"); - List> res = executeSql("SELECT MIN(b), MAX(b) FROM tbl"); - - assertEquals(1, res.size()); - assertEquals(2, res.get(0).size()); - assertTrue(Objects.deepEquals(new byte[] {1, 2, 3}, res.get(0).get(0))); - assertTrue(Objects.deepEquals(new byte[] {4, 5, 6}, res.get(0).get(1))); - } - finally { - executeSql("DROP TABLE IF EXISTS tbl"); - } + executeSql("CREATE TABLE tbl(b varbinary) WITH " + atomicity()); + executeSql("INSERT INTO tbl VALUES (NULL)"); + executeSql("INSERT INTO tbl VALUES (x'010203')"); + executeSql("INSERT INTO tbl VALUES (x'040506')"); + List> res = executeSql("SELECT MIN(b), MAX(b) FROM tbl"); + + assertEquals(1, res.size()); + assertEquals(2, res.get(0).size()); + assertTrue(Objects.deepEquals(new byte[]{1, 2, 3}, res.get(0).get(0))); + assertTrue(Objects.deepEquals(new byte[]{4, 5, 6}, res.get(0).get(1))); } /** */ @Test public void testBinaryConcat() { - try { - executeSql("CREATE TABLE tbl(b varbinary)"); - executeSql("INSERT INTO tbl VALUES (x'010203')"); - List> res = executeSql("SELECT b || x'040506' FROM tbl"); - - assertEquals(1, res.size()); - assertEquals(1, res.get(0).size()); - assertTrue(Objects.deepEquals(new byte[] {1, 2, 3, 4, 5, 6}, res.get(0).get(0))); - } - finally { - executeSql("DROP TABLE IF EXISTS tbl"); - } + executeSql("CREATE TABLE tbl(b varbinary) WITH " + atomicity()); + executeSql("INSERT INTO tbl VALUES (x'010203')"); + List> res = executeSql("SELECT b || x'040506' FROM tbl"); + + assertEquals(1, res.size()); + assertEquals(1, res.get(0).size()); + assertTrue(Objects.deepEquals(new byte[]{1, 2, 3, 4, 5, 6}, res.get(0).get(0))); } /** */ @Test public void testDecimalScale() { - sql("CREATE TABLE t (id INT PRIMARY KEY, val1 DECIMAL(5, 3), val2 DECIMAL(3), val3 DECIMAL)"); + sql("CREATE TABLE t (id INT PRIMARY KEY, val1 DECIMAL(5, 3), val2 DECIMAL(3), val3 DECIMAL) WITH " + atomicity()); // Check literals scale. sql("INSERT INTO t values (0, 0, 0, 0)"); @@ -482,15 +453,17 @@ public void testIsNotDistinctFromTypeConversion() { SqlTypeName[] numerics = new SqlTypeName[] {SqlTypeName.TINYINT, SqlTypeName.SMALLINT, SqlTypeName.INTEGER, SqlTypeName.BIGINT, SqlTypeName.DECIMAL, SqlTypeName.FLOAT, SqlTypeName.DOUBLE}; - sql("CREATE TABLE t1(key1 INTEGER, i1idx INTEGER, i1 INTEGER, chr1 VARCHAR, PRIMARY KEY(key1))"); - sql("CREATE INDEX t1_idx ON t1(i1idx)"); - sql("INSERT INTO t1 VALUES (1, 1, null, '1'), (2, 2, 2, '22'), (3, 33, 3, null), (4, null, 4, '4')"); - for (SqlTypeName type : numerics) { String t = type.getName(); - sql("CREATE TABLE t2(key2 " + t + ", i2idx " + t + ", i2 " + t + ", i3 INTEGER, chr2 VARCHAR, PRIMARY KEY(key2))"); + sql("CREATE TABLE t1(key1 INTEGER, i1idx INTEGER, i1 INTEGER, chr1 VARCHAR, PRIMARY KEY(key1)) WITH " + atomicity()); + sql("CREATE INDEX t1_idx ON t1(i1idx)"); + + sql("CREATE TABLE t2(key2 " + t + ", i2idx " + t + ", i2 " + t + ", i3 INTEGER, chr2 VARCHAR, PRIMARY KEY(key2)) " + + "WITH " + atomicity()); sql("CREATE INDEX t2_idx ON t2(i2idx)"); + + sql("INSERT INTO t1 VALUES (1, 1, null, '1'), (2, 2, 2, '22'), (3, 33, 3, null), (4, null, 4, '4')"); sql("INSERT INTO t2 VALUES (0, 0, 0, null, '0'), (11, null, 1, 1, '1'), (2, 2, 2, 2, '22'), (3, 3, null, 3, null)"); for (HintDefinition hint : Arrays.asList(HintDefinition.MERGE_JOIN, HintDefinition.NL_JOIN, HintDefinition.CNL_JOIN)) { @@ -571,14 +544,19 @@ public void testIsNotDistinctFromTypeConversion() { .check(); } + clearTransaction(); + + sql("DROP TABLE t1"); sql("DROP TABLE t2"); + + clearTransaction(); } } /** */ @Test public void testNumericConversion() { - sql("CREATE TABLE t (v1 TINYINT, v2 SMALLINT, v3 INT, v4 BIGINT, v5 DECIMAL, v6 FLOAT, v7 DOUBLE)"); + sql("CREATE TABLE t (v1 TINYINT, v2 SMALLINT, v3 INT, v4 BIGINT, v5 DECIMAL, v6 FLOAT, v7 DOUBLE) WITH " + atomicity()); List params = F.asList((byte)1, (short)2, 3, 4L, BigDecimal.valueOf(5), 6f, 7d); @@ -599,6 +577,8 @@ public void testNumericConversion() { /** */ @Test public void testFunctionArgsToNumericImplicitConversion() { + assumeTrue("Test use queries that doesn't touch any data. Skip for tx modes", sqlTxMode == SqlTransactionMode.NONE); + assertQuery("select decode(?, 0, 0, 1, 1.0)").withParams(0).returns(new BigDecimal("0.0")).check(); assertQuery("select decode(?, 0, 0, 1, 1.0)").withParams(1).returns(new BigDecimal("1.0")).check(); assertQuery("select decode(?, 0, 0, 1, 1.000)").withParams(0).returns(new BigDecimal("0.000")).check(); @@ -672,6 +652,8 @@ public void testArithmeticOverflow() { /** */ @Test public void testCastDecimalOverflows() { + assumeTrue("Test use queries that doesn't touch any data. Skip for tx modes", sqlTxMode == SqlTransactionMode.NONE); + // BIGINT assertQuery("SELECT CAST(9223372036854775807.1 AS BIGINT)").returns(9223372036854775807L).check(); assertQuery("SELECT CAST(9223372036854775807.9 AS BIGINT)").returns(9223372036854775807L).check(); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DateTimeTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DateTimeTest.java index 1335e33aa6e20..2bf25ef6a18ae 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DateTimeTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/DateTimeTest.java @@ -31,13 +31,13 @@ import static java.util.Collections.singletonList; /** */ -public class DateTimeTest extends AbstractBasicIntegrationTest { +public class DateTimeTest extends AbstractBasicIntegrationTransactionalTest { /** */ private static final SimpleDateFormat DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS"); /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); + @Override protected void init() throws Exception { + super.init(); QueryEntity qryEnt = new QueryEntity(); qryEnt.setKeyFieldName("ID"); @@ -51,22 +51,22 @@ public class DateTimeTest extends AbstractBasicIntegrationTest { qryEnt.addQueryField("SQLTIMESTAMP", Timestamp.class.getName(), null); qryEnt.setTableName("datetimetable"); - final CacheConfiguration cfg = new CacheConfiguration<>(qryEnt.getTableName()); - - cfg.setCacheMode(CacheMode.PARTITIONED) + final CacheConfiguration cfg = this.cacheConfiguration() + .setName(qryEnt.getTableName()) + .setCacheMode(CacheMode.PARTITIONED) .setBackups(1) .setQueryEntities(singletonList(qryEnt)) .setSqlSchema("PUBLIC"); IgniteCache dateTimeCache = client.createCache(cfg); - dateTimeCache.put(1, new DateTimeEntry(1, javaDate("2020-10-01 12:00:00.000"), + put(client, dateTimeCache, 1, new DateTimeEntry(1, javaDate("2020-10-01 12:00:00.000"), sqlDate("2020-10-01"), sqlTime("12:00:00"), sqlTimestamp("2020-10-01 12:00:00.000"))); - dateTimeCache.put(2, new DateTimeEntry(2, javaDate("2020-12-01 00:10:20.000"), + put(client, dateTimeCache, 2, new DateTimeEntry(2, javaDate("2020-12-01 00:10:20.000"), sqlDate("2020-12-01"), sqlTime("00:10:20"), sqlTimestamp("2020-12-01 00:10:20.000"))); - dateTimeCache.put(3, new DateTimeEntry(3, javaDate("2020-10-20 13:15:00.000"), + put(client, dateTimeCache, 3, new DateTimeEntry(3, javaDate("2020-10-20 13:15:00.000"), sqlDate("2020-10-20"), sqlTime("13:15:00"), sqlTimestamp("2020-10-20 13:15:00.000"))); - dateTimeCache.put(4, new DateTimeEntry(4, javaDate("2020-01-01 22:40:00.000"), + put(client, dateTimeCache, 4, new DateTimeEntry(4, javaDate("2020-01-01 22:40:00.000"), sqlDate("2020-01-01"), sqlTime("22:40:00"), sqlTimestamp("2020-01-01 22:40:00.000"))); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/HashSpoolIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/HashSpoolIntegrationTest.java index d1ddc5423bf5a..605e1175687d6 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/HashSpoolIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/HashSpoolIntegrationTest.java @@ -23,11 +23,11 @@ /** * Hash spool test. */ -public class HashSpoolIntegrationTest extends AbstractBasicIntegrationTest { +public class HashSpoolIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ @Test public void testNullsInSearchRow() { - executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER)"); + executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER) WITH " + atomicity()); executeSql("INSERT INTO t VALUES (null, 0), (1, 1), (2, 2), (3, null)"); assertQuery("SELECT i1, (SELECT i2 FROM t WHERE i1=t1.i1) FROM t t1") @@ -50,8 +50,8 @@ public void testNullsInSearchRow() { /** */ @Test public void testNullsInSearchRowMultipleColumns() { - executeSql("CREATE TABLE t0(i1 INTEGER, i2 INTEGER)"); - executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER)"); + executeSql("CREATE TABLE t0(i1 INTEGER, i2 INTEGER) WITH " + atomicity()); + executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER) WITH " + atomicity()); executeSql("INSERT INTO t0 VALUES (null, 0), (1, null), (null, 2), (3, null), (1, 1)"); executeSql("INSERT INTO t1 VALUES (null, 0), (null, 1), (2, null), (3, null), (1, 1)"); @@ -67,7 +67,7 @@ public void testNullsInSearchRowMultipleColumns() { /** */ @Test public void testHashSpoolCondition() { - executeSql("CREATE TABLE t(i INTEGER)"); + executeSql("CREATE TABLE t(i INTEGER) WITH " + atomicity()); executeSql("INSERT INTO t VALUES (0), (1), (2)"); String sql = "SELECT i, (SELECT i FROM t WHERE i=t1.i AND i-1=0) FROM t AS t1"; @@ -83,10 +83,10 @@ public void testHashSpoolCondition() { /** */ @Test public void testIsNotDistinctFrom() { - executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER)"); - executeSql("INSERT INTO t1 VALUES (1, null), (2, 2), (null, 3), (3, null)"); + executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER) WITH " + atomicity()); + executeSql("CREATE TABLE t2(i3 INTEGER, i4 INTEGER) WITH " + atomicity()); - executeSql("CREATE TABLE t2(i3 INTEGER, i4 INTEGER)"); + executeSql("INSERT INTO t1 VALUES (1, null), (2, 2), (null, 3), (3, null)"); executeSql("INSERT INTO t2 VALUES (1, 1), (2, 2), (null, 3), (4, null)"); String sql = "SELECT /*+ CNL_JOIN */ i1, i4 FROM t1 JOIN t2 ON i1 IS NOT DISTINCT FROM i3"; diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexMultiRangeScanIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexMultiRangeScanIntegrationTest.java index e59be725daf73..a4239b8f33a0f 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexMultiRangeScanIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexMultiRangeScanIntegrationTest.java @@ -16,7 +16,9 @@ */ package org.apache.ignite.internal.processors.query.calcite.integration; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.List; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; import org.apache.ignite.internal.util.typedef.F; @@ -29,26 +31,34 @@ * Tests index multi-range scans (with SEARCH/SARG operator or with dynamic parameters). */ @RunWith(Parameterized.class) -public class IndexMultiRangeScanIntegrationTest extends AbstractBasicIntegrationTest { +public class IndexMultiRangeScanIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ - @Parameterized.Parameter + @Parameterized.Parameter(1) public boolean dynamicParams; - /** */ - @Parameterized.Parameters(name = "dynamicParams={0}") - public static List parameters() { - return F.asList(true, false); + /** @return Test parameters. */ + @Parameterized.Parameters(name = "sqlTxMode={0},dynamicParams={1}") + public static Collection parameters() { + List params = new ArrayList<>(); + + for (boolean dynamicParams : new boolean[]{true, false}) { + for (SqlTransactionMode sqlTxMode : SqlTransactionMode.values()) { + params.add(new Object[]{sqlTxMode, dynamicParams}); + } + } + + return params; } /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); + @Override protected void init() throws Exception { + super.init(); - sql("CREATE TABLE test (c1 INTEGER, c2 VARCHAR, c3 INTEGER)"); + sql("CREATE TABLE test (c1 INTEGER, c2 VARCHAR, c3 INTEGER) WITH " + atomicity()); sql("CREATE INDEX c1c2c3 ON test(c1, c2, c3)"); - sql("CREATE TABLE test_desc (c1 INTEGER, c2 VARCHAR, c3 INTEGER)"); + sql("CREATE TABLE test_desc (c1 INTEGER, c2 VARCHAR, c3 INTEGER) WITH " + atomicity()); sql("CREATE INDEX c1c2c3_desc ON test_desc(c1 DESC, c2 DESC, c3 DESC)"); - sql("CREATE TABLE test_pk (c1 INTEGER, c2 VARCHAR, c3 INTEGER, PRIMARY KEY(c1, c2, c3))"); + sql("CREATE TABLE test_pk (c1 INTEGER, c2 VARCHAR, c3 INTEGER, PRIMARY KEY(c1, c2, c3)) WITH " + atomicity()); for (String tbl : F.asList("test", "test_desc", "test_pk")) { sql("INSERT INTO " + tbl + "(c1, c2, c3) VALUES (0, null, 0)"); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java index 7e7c0df7cbe4e..df79c0490c098 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/IndexScanlIntegrationTest.java @@ -31,7 +31,6 @@ import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; import org.apache.ignite.internal.processors.query.calcite.exec.ExecutionContext; @@ -51,7 +50,7 @@ /** * Index scan test. */ -public class IndexScanlIntegrationTest extends AbstractBasicIntegrationTest { +public class IndexScanlIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ private static final int ROWS_CNT = 100; @@ -63,7 +62,7 @@ public class IndexScanlIntegrationTest extends AbstractBasicIntegrationTest { /** */ @Test public void testNullsInCNLJSearchRow() { - executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER) WITH TEMPLATE=REPLICATED "); + executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER) WITH TEMPLATE=REPLICATED," + atomicity()); executeSql("INSERT INTO t VALUES (0, null), (1, null), (2, 2), (3, null), (4, null), (null, 5)"); executeSql("CREATE INDEX t_idx ON t(i1)"); @@ -90,7 +89,7 @@ public void testNullsInCNLJSearchRow() { /** */ @Test public void testNullsInSearchRow() { - executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER) WITH TEMPLATE=REPLICATED "); + executeSql("CREATE TABLE t(i1 INTEGER, i2 INTEGER) WITH TEMPLATE=REPLICATED," + atomicity()); executeSql("INSERT INTO t VALUES (null, 0), (1, null), (2, 2), (3, null)"); executeSql("CREATE INDEX t_idx ON t(i1, i2)"); @@ -127,7 +126,8 @@ public void testNullsInSearchRow() { /** */ @Test public void testSegmentedIndexes() { - IgniteCache emp = client.getOrCreateCache(new CacheConfiguration("emp") + IgniteCache emp = client.getOrCreateCache(this.cacheConfiguration() + .setName("emp") .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class).setTableName("emp"))) .setQueryParallelism(10) @@ -165,7 +165,7 @@ public void testSegmentedIndexes() { /** */ @Test public void testScanBooleanField() { - executeSql("CREATE TABLE t(i INTEGER, b BOOLEAN)"); + executeSql("CREATE TABLE t(i INTEGER, b BOOLEAN) WITH " + atomicity()); executeSql("INSERT INTO t VALUES (0, TRUE), (1, TRUE), (2, FALSE), (3, FALSE), (4, null)"); executeSql("CREATE INDEX t_idx ON t(b)"); @@ -218,11 +218,12 @@ public void testScanBooleanField() { /** */ @Test public void testIsNotDistinctFrom() { - executeSql("CREATE TABLE t1(i1 INTEGER) WITH TEMPLATE=REPLICATED"); - executeSql("INSERT INTO t1 VALUES (1), (2), (null), (3)"); + executeSql("CREATE TABLE t1(i1 INTEGER) WITH TEMPLATE=REPLICATED," + atomicity()); + executeSql("CREATE TABLE t2(i2 INTEGER, i3 INTEGER) WITH TEMPLATE=REPLICATED," + atomicity()); - executeSql("CREATE TABLE t2(i2 INTEGER, i3 INTEGER) WITH TEMPLATE=REPLICATED"); + executeSql("INSERT INTO t1 VALUES (1), (2), (null), (3)"); executeSql("INSERT INTO t2 VALUES (1, 1), (2, 2), (null, 3), (4, null)"); + executeSql("CREATE INDEX t2_idx ON t2(i2)"); String sql = "SELECT /*+ CNL_JOIN */ i1, i3 FROM t1 JOIN t2 ON i1 IS NOT DISTINCT FROM i2"; @@ -262,7 +263,7 @@ public void testInlineScan() { checkSingleColumnInlineScan(false, "DECIMAL", BigDecimal::valueOf); // Multi columns scans. - executeSql("CREATE TABLE t(id INTEGER PRIMARY KEY, i1 INTEGER, i2 INTEGER, i3 INTEGER)"); + executeSql("CREATE TABLE t(id INTEGER PRIMARY KEY, i1 INTEGER, i2 INTEGER, i3 INTEGER) WITH " + atomicity()); executeSql("CREATE INDEX t_idx ON t(i1, i3)"); RowCountingIndex idx = injectRowCountingIndex(grid(0), "T", "T_IDX"); @@ -277,7 +278,7 @@ public void testInlineScan() { /** */ public void checkSingleColumnInlineScan(boolean expInline, String dataType, IntFunction valFactory) { - executeSql("CREATE TABLE t(id INTEGER PRIMARY KEY, val " + dataType + ')'); + executeSql("CREATE TABLE t(id INTEGER PRIMARY KEY, val " + dataType + ") WITH " + atomicity()); try { executeSql("CREATE INDEX t_idx ON t(val)"); @@ -294,8 +295,14 @@ public void checkSingleColumnInlineScan(boolean expInline, String dataType, IntF if (expInline) { checker.matches(QueryChecker.containsIndexScan("PUBLIC", "T", "T_IDX")).check(); - assertEquals(ROWS_CNT, idx.rowsProcessed()); - assertTrue(idx.isInlineScan()); + if (sqlTxMode == SqlTransactionMode.NONE) { + assertEquals(ROWS_CNT, idx.rowsProcessed()); + assertTrue(idx.isInlineScan()); + } + else if (sqlTxMode == SqlTransactionMode.ALL) { + assertEquals(0, idx.rowsProcessed()); + assertFalse(idx.isInlineScan()); + } } else { checker.check(); @@ -304,6 +311,7 @@ public void checkSingleColumnInlineScan(boolean expInline, String dataType, IntF } } finally { + clearTransaction(); executeSql("DROP TABLE t"); } } @@ -323,8 +331,10 @@ public void checkMultiColumnsInlineScan( if (expInline) { checker.matches(QueryChecker.containsIndexScan("PUBLIC", "T", "T_IDX")).check(); - assertEquals(ROWS_CNT, idx.rowsProcessed()); - assertTrue(idx.isInlineScan()); + if (sqlTxMode == SqlTransactionMode.NONE) { + assertEquals(ROWS_CNT, idx.rowsProcessed()); + assertTrue(idx.isInlineScan()); + } } else { checker.check(); @@ -336,11 +346,12 @@ public void checkMultiColumnsInlineScan( /** */ @Test public void testNoIndexHint() { - executeSql("CREATE TABLE t1(i1 INTEGER) WITH TEMPLATE=PARTITIONED"); - executeSql("INSERT INTO t1 VALUES (1), (2), (30), (40)"); + executeSql("CREATE TABLE t1(i1 INTEGER) WITH TEMPLATE=PARTITIONED," + atomicity()); executeSql("CREATE INDEX t1_idx ON t1(i1)"); - executeSql("CREATE TABLE t2(i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED"); + executeSql("CREATE TABLE t2(i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED," + atomicity()); + + executeSql("INSERT INTO t1 VALUES (1), (2), (30), (40)"); for (int i = 0; i < 100; ++i) executeSql("INSERT INTO t2 VALUES (?, ?)", i, i); @@ -370,12 +381,18 @@ public void testNoIndexHint() { /** */ @Test public void testForcedIndexHint() { - executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED"); + executeSql("CREATE TABLE t1(i1 INTEGER, i2 INTEGER, i3 INTEGER) WITH TEMPLATE=PARTITIONED," + atomicity()); executeSql("CREATE INDEX t1_idx1 ON t1(i1)"); executeSql("CREATE INDEX t1_idx2 ON t1(i2)"); executeSql("CREATE INDEX t1_idx3 ON t1(i3)"); + executeSql("CREATE TABLE t2(i21 INTEGER, i22 INTEGER, i23 INTEGER) WITH TEMPLATE=PARTITIONED," + atomicity()); + + executeSql("CREATE INDEX t2_idx1 ON t2(i21)"); + executeSql("CREATE INDEX t2_idx2 ON t2(i22)"); + executeSql("CREATE INDEX t2_idx3 ON t2(i23)"); + executeSql("INSERT INTO t1 VALUES (1, 2, 3)"); assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX1) */ i1 FROM t1 where i1=1 and i2=2 and i3=3") @@ -393,15 +410,9 @@ public void testForcedIndexHint() { .returns(1) .check(); - executeSql("CREATE TABLE t2(i21 INTEGER, i22 INTEGER, i23 INTEGER) WITH TEMPLATE=PARTITIONED"); - for (int i = 99; i < 300; ++i) executeSql("INSERT INTO t2 VALUES (?, ?, ?)", i + 1, i + 1, i + 1); - executeSql("CREATE INDEX t2_idx1 ON t2(i21)"); - executeSql("CREATE INDEX t2_idx2 ON t2(i22)"); - executeSql("CREATE INDEX t2_idx3 ON t2(i23)"); - assertQuery("SELECT /*+ FORCE_INDEX(T1_IDX2), FORCE_INDEX(T2_IDX2) */ i1, i22 FROM t1, t2 where i2=i22 " + "and i3=i23 + 1") .matches(QueryChecker.containsIndexScan("PUBLIC", "T1", "T1_IDX2")) diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java index d41bb01eb58ef..603a7123b363b 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinIntegrationTest.java @@ -16,6 +16,7 @@ */ package org.apache.ignite.internal.processors.query.calcite.integration; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; @@ -27,25 +28,31 @@ /** */ @RunWith(Parameterized.class) -public class JoinIntegrationTest extends AbstractBasicIntegrationTest { +public class JoinIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ - @Parameterized.Parameter + @Parameterized.Parameter(1) public JoinType joinType; /** */ - @Parameterized.Parameters(name = "joinType={0}") + @Parameterized.Parameters(name = "sqlTxMode={0},joinType={1}") public static List params() { - return Arrays.stream(JoinType.values()) - .map(jt -> new Object[]{jt}) - .collect(Collectors.toList()); + List params = new ArrayList<>(); + + for (SqlTransactionMode sqlTxMode : SqlTransactionMode.values()) { + for (JoinType jt : JoinType.values()) { + params.add(new Object[]{sqlTxMode, jt}); + } + } + + return params; } /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); + @Override protected void init() throws Exception { + super.init(); - executeSql("create table t1 (c1 int, c2 int, c3 int)"); - executeSql("create table t2 (c1 int, c2 int, c3 int)"); + executeSql("create table t1 (c1 int, c2 int, c3 int) WITH " + atomicity()); + executeSql("create table t2 (c1 int, c2 int, c3 int) WITH " + atomicity()); executeSql("create index t1_idx on t1 (c3, c2, c1)"); executeSql("create index t2_idx on t2 (c3, c2, c1)"); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinRehashIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinRehashIntegrationTest.java index 8b6c46aba791a..2370eea664f0c 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinRehashIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/JoinRehashIntegrationTest.java @@ -23,7 +23,7 @@ import org.junit.Test; /** */ -public class JoinRehashIntegrationTest extends AbstractBasicIntegrationTest { +public class JoinRehashIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** {@inheritDoc} */ @Override protected int nodeCount() { return 3; @@ -81,13 +81,13 @@ private void prepareTables() { " price decimal,\n" + " amount int,\n" + " PRIMARY KEY (id))\n" + - " WITH \"cache_name=order_items,backups=1\""); + " WITH \"cache_name=order_items,backups=1," + atomicity() + "\""); sql("CREATE TABLE orders (\n" + " id int,\n" + " region varchar,\n" + " PRIMARY KEY (id))\n" + - " WITH \"cache_name=orders,backups=1\""); + " WITH \"cache_name=orders,backups=1," + atomicity() + "\""); sql("CREATE INDEX order_items_orderId ON order_items (orderId ASC)"); sql("CREATE INDEX orders_region ON orders (region ASC)"); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/KeepBinaryIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/KeepBinaryIntegrationTest.java index f3aff6c99c288..bf62dd6b83370 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/KeepBinaryIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/KeepBinaryIntegrationTest.java @@ -25,7 +25,6 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.util.typedef.F; import org.junit.Test; @@ -33,7 +32,7 @@ /** * Test "keep binary" in cache queries. */ -public class KeepBinaryIntegrationTest extends AbstractBasicIntegrationTest { +public class KeepBinaryIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ private static final String CACHE_NAME = "cache"; @@ -41,7 +40,7 @@ public class KeepBinaryIntegrationTest extends AbstractBasicIntegrationTest { @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration()); - cfg.setCacheConfiguration(new CacheConfiguration<>(CACHE_NAME).setIndexedTypes(Integer.class, Person.class)); + cfg.setCacheConfiguration(cacheConfiguration().setName(CACHE_NAME).setIndexedTypes(Integer.class, Person.class)); return cfg; } @@ -58,35 +57,44 @@ public void testKeepBinary() { Person p2 = new Person(2, "name2", F.asList(new Person(3, "name3", null), new Person(4, "name4", null))); - cache.put(0, p0); - cache.put(1, p1); - cache.put(2, p2); - - List> res = cache.query(new SqlFieldsQuery("SELECT _VAL, obj, name FROM Person ORDER BY id")).getAll(); - - assertEquals(3, res.size()); - assertEquals(p0, res.get(0).get(0)); - assertEquals(p1, res.get(1).get(0)); - assertEquals(p2, res.get(2).get(0)); - assertEquals(p0.obj, res.get(0).get(1)); - assertEquals(p1.obj, res.get(1).get(1)); - assertEquals(p2.obj, res.get(2).get(1)); - assertEquals(p0.name, res.get(0).get(2)); - assertEquals(p1.name, res.get(1).get(2)); - assertEquals(p2.name, res.get(2).get(2)); - - res = cache.withKeepBinary().query(new SqlFieldsQuery("SELECT _VAL, obj, name FROM Person ORDER BY id")).getAll(); - - assertEquals(3, res.size()); - assertTrue(res.get(0).get(0) instanceof BinaryObject); - assertTrue(res.get(1).get(0) instanceof BinaryObject); - assertTrue(res.get(1).get(0) instanceof BinaryObject); - assertNull(res.get(0).get(1)); - assertTrue(res.get(1).get(1) instanceof BinaryObject); - assertTrue(res.get(1).get(1) instanceof BinaryObject); - assertEquals(p0.name, res.get(0).get(2)); - assertEquals(p1.name, res.get(1).get(2)); - assertEquals(p2.name, res.get(2).get(2)); + put(client, cache, 0, p0); + put(client, cache, 1, p1); + put(client, cache, 2, p2); + + SupplierX checker = () -> { + List> res = cache.query(new SqlFieldsQuery("SELECT _VAL, obj, name FROM Person ORDER BY id")).getAll(); + + assertEquals(3, res.size()); + assertEquals(p0, res.get(0).get(0)); + assertEquals(p1, res.get(1).get(0)); + assertEquals(p2, res.get(2).get(0)); + assertEquals(p0.obj, res.get(0).get(1)); + assertEquals(p1.obj, res.get(1).get(1)); + assertEquals(p2.obj, res.get(2).get(1)); + assertEquals(p0.name, res.get(0).get(2)); + assertEquals(p1.name, res.get(1).get(2)); + assertEquals(p2.name, res.get(2).get(2)); + + res = cache.withKeepBinary().query(new SqlFieldsQuery("SELECT _VAL, obj, name FROM Person ORDER BY id")).getAll(); + + assertEquals(3, res.size()); + assertTrue(res.get(0).get(0) instanceof BinaryObject); + assertTrue(res.get(1).get(0) instanceof BinaryObject); + assertTrue(res.get(1).get(0) instanceof BinaryObject); + assertNull(res.get(0).get(1)); + assertTrue(res.get(1).get(1) instanceof BinaryObject); + assertTrue(res.get(1).get(1) instanceof BinaryObject); + assertEquals(p0.name, res.get(0).get(2)); + assertEquals(p1.name, res.get(1).get(2)); + assertEquals(p2.name, res.get(2).get(2)); + + return null; + }; + + if (sqlTxMode == SqlTransactionMode.NONE) + checker.get(); + else + txAction(client, checker); } /** */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LimitOffsetIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LimitOffsetIntegrationTest.java index 71c40bc818ca3..89b2d73ffdab9 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LimitOffsetIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LimitOffsetIntegrationTest.java @@ -24,7 +24,6 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.calcite.exec.rel.AbstractNode; @@ -38,7 +37,7 @@ /** * Limit / offset tests. */ -public class LimitOffsetIntegrationTest extends AbstractBasicIntegrationTest { +public class LimitOffsetIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ private static IgniteCache cacheRepl; @@ -46,7 +45,9 @@ public class LimitOffsetIntegrationTest extends AbstractBasicIntegrationTest { private static IgniteCache cachePart; /** {@inheritDoc} */ - @Override protected void beforeTest() throws Exception { + @Override protected void init() throws Exception { + super.init(); + cacheRepl = client.cache("TEST_REPL"); cachePart = client.cache("TEST_PART"); } @@ -78,11 +79,13 @@ public class LimitOffsetIntegrationTest extends AbstractBasicIntegrationTest { return super.getConfiguration(igniteInstanceName) .setCacheConfiguration( - new CacheConfiguration<>(eRepl.getTableName()) + cacheConfiguration() + .setName(eRepl.getTableName()) .setCacheMode(CacheMode.REPLICATED) .setQueryEntities(singletonList(eRepl)) .setSqlSchema("PUBLIC"), - new CacheConfiguration<>(ePart.getTableName()) + cacheConfiguration() + .setName(ePart.getTableName()) .setCacheMode(CacheMode.PARTITIONED) .setQueryEntities(singletonList(ePart)) .setSqlSchema("PUBLIC")); @@ -188,10 +191,12 @@ public void testOffsetOutOfRange() throws Exception { * @param rows Rows count. */ private void fillCache(IgniteCache c, int rows) throws InterruptedException { + clearTransaction(); + c.clear(); for (int i = 0; i < rows; ++i) - c.put(i, "val_" + String.format("%05d", i)); + put(client, c, i, "val_" + String.format("%05d", i)); awaitPartitionMapExchange(); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LocalDateTimeSupportTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LocalDateTimeSupportTest.java index 6bda9cf207521..1b1f5bf7d529a 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LocalDateTimeSupportTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/LocalDateTimeSupportTest.java @@ -31,8 +31,8 @@ import java.util.List; import java.util.function.Function; import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor; @@ -48,130 +48,133 @@ /** */ @RunWith(Parameterized.class) -public class LocalDateTimeSupportTest extends AbstractBasicIntegrationTest { +public class LocalDateTimeSupportTest extends AbstractBasicIntegrationTransactionalTest { /** */ - @Parameterized.Parameter() + @Parameterized.Parameter(1) public boolean isValidationEnabled; /** */ - @Parameterized.Parameter(1) + @Parameterized.Parameter(2) public String sqlType; /** */ - @Parameterized.Parameter(2) + @Parameterized.Parameter(3) public Class colType; /** */ - @Parameterized.Parameter(3) + @Parameterized.Parameter(4) public Class objType; /** */ - @Parameterized.Parameter(4) + @Parameterized.Parameter(5) public Function sqlTypeConverter; /** */ - @Parameterized.Parameter(5) + @Parameterized.Parameter(6) public boolean isOldDate; /** */ - @Parameterized.Parameters(name = "isValidationEnabled={0}, sqlType={1}, columnCls={2}, testObjCls={3}, beforeGregorian={5}") + @Parameterized.Parameters( + name = "sqlTxMode={0}, isValidationEnabled={1}, sqlType={2}, columnCls={3}, testObjCls={4}, beforeGregorian={6}") public static Collection parameters() { Collection params = new ArrayList<>(); - for (boolean isOldDate: Arrays.asList(true, false)) { - for (boolean isV : Arrays.asList(true, false)) { - params.add(new Object[] { - isV, "TIMESTAMP", null, LocalDateTime.class, f(ts -> convertToTimestamp((LocalDateTime)ts)), isOldDate - }); - params.add(new Object[] { - isV, "TIMESTAMP", null, Date.class, f(ts -> new Timestamp(((Date)ts).getTime())), isOldDate - }); - params.add(new Object[] { - isV, "TIMESTAMP", null, java.sql.Date.class, f(ts -> new Timestamp(((Date)ts).getTime())), isOldDate - }); - - for (Class testObjCls : Arrays.asList(Timestamp.class, LocalDateTime.class, Date.class, java.sql.Date.class)) { + for (SqlTransactionMode sqlTxmode : SqlTransactionMode.values()) { + for (boolean isOldDate: Arrays.asList(true, false)) { + for (boolean isV : Arrays.asList(true, false)) { params.add(new Object[] { - isV, null, Timestamp.class, testObjCls, f(ts -> { - if (ts instanceof LocalDateTime) - return convertToTimestamp((LocalDateTime)ts); - return ts; - }), - isOldDate + sqlTxmode, isV, "TIMESTAMP", null, LocalDateTime.class, f(ts -> convertToTimestamp((LocalDateTime)ts)), isOldDate }); - params.add(new Object[] { - isV, null, Date.class, testObjCls, f(ts -> { - if (testObjCls == LocalDateTime.class) - return new Date(convertToTimestamp((LocalDateTime)ts).getTime()); - else if (testObjCls == Timestamp.class) - return new Date(((Timestamp)ts).getTime()); - return ts; - }), - isOldDate + sqlTxmode, isV, "TIMESTAMP", null, Date.class, f(ts -> new Timestamp(((Date)ts).getTime())), isOldDate }); - params.add(new Object[] { - isV, null, LocalDateTime.class, testObjCls, f(ts -> { - if (testObjCls == Timestamp.class) - return ((Timestamp)ts).toLocalDateTime(); - else if (testObjCls == java.util.Date.class) - return new Timestamp(((Date)ts).getTime()).toLocalDateTime(); - else if (testObjCls == java.sql.Date.class) - return ((java.sql.Date)ts).toLocalDate().atStartOfDay(); - else - return ts; - }), - isOldDate + sqlTxmode, isV, "TIMESTAMP", null, java.sql.Date.class, f(ts -> new Timestamp(((Date)ts).getTime())), isOldDate }); - } - - params.add(new Object[] { - isV, "DATE", null, LocalDate.class, f(d -> convertToSqlDate((LocalDate)d)), isOldDate - }); - for (Class testObjCls : Arrays.asList(LocalDate.class, java.sql.Date.class)) { - params.add(new Object[] { - isV, null, java.sql.Date.class, testObjCls, f(ts -> { - if (testObjCls == LocalDate.class) - return convertToSqlDate((LocalDate)ts); - return ts; - }), - isOldDate - }); + for (Class testObjCls : Arrays.asList(Timestamp.class, LocalDateTime.class, Date.class, java.sql.Date.class)) { + params.add(new Object[] { + sqlTxmode, isV, null, Timestamp.class, testObjCls, f(ts -> { + if (ts instanceof LocalDateTime) + return convertToTimestamp((LocalDateTime)ts); + return ts; + }), + isOldDate + }); + + params.add(new Object[] { + sqlTxmode, isV, null, Date.class, testObjCls, f(ts -> { + if (testObjCls == LocalDateTime.class) + return new Date(convertToTimestamp((LocalDateTime)ts).getTime()); + else if (testObjCls == Timestamp.class) + return new Date(((Timestamp)ts).getTime()); + return ts; + }), + isOldDate + }); + + params.add(new Object[] { + sqlTxmode, isV, null, LocalDateTime.class, testObjCls, f(ts -> { + if (testObjCls == Timestamp.class) + return ((Timestamp)ts).toLocalDateTime(); + else if (testObjCls == java.util.Date.class) + return new Timestamp(((Date)ts).getTime()).toLocalDateTime(); + else if (testObjCls == java.sql.Date.class) + return ((java.sql.Date)ts).toLocalDate().atStartOfDay(); + else + return ts; + }), + isOldDate + }); + } params.add(new Object[] { - isV, null, LocalDate.class, testObjCls, f(ts -> { - if (testObjCls == java.sql.Date.class) - return ((java.sql.Date)ts).toLocalDate(); - return ts; - }), - isOldDate + sqlTxmode, isV, "DATE", null, LocalDate.class, f(d -> convertToSqlDate((LocalDate)d)), isOldDate }); - } - params.add(new Object[] { - isV, "TIME", null, LocalTime.class, f(t -> convertToSqlTime((LocalTime)t)), isOldDate - }); + for (Class testObjCls : Arrays.asList(LocalDate.class, java.sql.Date.class)) { + params.add(new Object[] { + sqlTxmode, isV, null, java.sql.Date.class, testObjCls, f(ts -> { + if (testObjCls == LocalDate.class) + return convertToSqlDate((LocalDate)ts); + return ts; + }), + isOldDate + }); + + params.add(new Object[] { + sqlTxmode, isV, null, LocalDate.class, testObjCls, f(ts -> { + if (testObjCls == java.sql.Date.class) + return ((java.sql.Date)ts).toLocalDate(); + return ts; + }), + isOldDate + }); + } - for (Class testObjCls : Arrays.asList(LocalTime.class, java.sql.Time.class)) { params.add(new Object[] { - isV, null, java.sql.Time.class, testObjCls, f(ts -> { - if (testObjCls == LocalTime.class) - return convertToSqlTime((LocalTime)ts); - return ts; - }), - isOldDate + sqlTxmode, isV, "TIME", null, LocalTime.class, f(t -> convertToSqlTime((LocalTime)t)), isOldDate }); - params.add(new Object[] { - isV, null, LocalTime.class, testObjCls, f(ts -> { - if (testObjCls == java.sql.Time.class) - return ((java.sql.Time)ts).toLocalTime(); - return ts; - }), - isOldDate - }); + for (Class testObjCls : Arrays.asList(LocalTime.class, java.sql.Time.class)) { + params.add(new Object[] { + sqlTxmode, isV, null, java.sql.Time.class, testObjCls, f(ts -> { + if (testObjCls == LocalTime.class) + return convertToSqlTime((LocalTime)ts); + return ts; + }), + isOldDate + }); + + params.add(new Object[] { + sqlTxmode, isV, null, LocalTime.class, testObjCls, f(ts -> { + if (testObjCls == java.sql.Time.class) + return ((java.sql.Time)ts).toLocalTime(); + return ts; + }), + isOldDate + }); + } } } } @@ -210,9 +213,10 @@ public void testTemporalTypes() { executeSql("INSERT INTO DATA(_key, id, data) values(?, ?, ?)", 0, 0, testObj); - grid(0).cache(DEFAULT_CACHE_NAME).put(1, new Data(1, testObj)); + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); - grid(0).cache(DEFAULT_CACHE_NAME).put(2, grid(0).binary().toBinary(new Data(2, testObj))); + put(client, cache, 1, new Data(1, testObj)); + put(client, cache, 2, client.binary().toBinary(new Data(2, testObj))); List> selectData = executeSql("SELECT data FROM DATA"); @@ -227,7 +231,8 @@ private void createTable() { executeSql("CREATE TABLE DATA (id INT PRIMARY KEY, data " + sqlType + ") WITH" + " \"KEY_TYPE=java.lang.Integer" + ", VALUE_TYPE=" + Data.class.getName() + - ", CACHE_NAME=default\""); + ", CACHE_NAME=default" + + ", " + atomicity() + "\""); } else { QueryEntity projEntity = new QueryEntity(); @@ -238,7 +243,8 @@ private void createTable() { projEntity.setTableName("DATA"); - grid(0).createCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME) + client.createCache(cacheConfiguration() + .setName(DEFAULT_CACHE_NAME) .setQueryEntities(singletonList(projEntity)) .setSqlSchema("PUBLIC")); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java index f69673b5ab8a8..6d1bdbb929696 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/ServerStatisticsIntegrationTest.java @@ -27,9 +27,9 @@ import java.util.Set; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.query.annotations.QuerySqlField; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.query.QueryEngine; import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor; @@ -44,7 +44,7 @@ /** * Tests for server side statistics usage. */ -public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTest { +public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** Server instance. */ private IgniteEx srv; @@ -86,10 +86,15 @@ public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTes }; /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - super.beforeTestsStarted(); + @Override protected void init() throws Exception { + super.init(); + + createAllTypesTable(); + + createAndPopulateTable(grid(0), 2, CacheMode.PARTITIONED); + + populateAllTypesTable(0, ROW_COUNT); - createAndPopulateAllTypesTable(0, ROW_COUNT); } /** {@inheritDoc} */ @@ -112,8 +117,6 @@ public class ServerStatisticsIntegrationTest extends AbstractBasicIntegrationTes public void testQueryCostWithStatistics() throws IgniteCheckedException { String sql = "select name from person where salary is not null"; - createAndPopulateTable(); - StatisticsKey key = new StatisticsKey("PUBLIC", "PERSON"); srv = ignite(0); @@ -421,8 +424,6 @@ public void testBorders() throws IgniteCheckedException { */ @Test public void testSizeIntOverflow() { - createAndPopulateTable(); - F.first(grid(0).context().cache().cache(TABLE_NAME).context().offheap().cacheDataStores()) .updateSize(CU.cacheId(TABLE_NAME), 1L + Integer.MAX_VALUE); @@ -449,7 +450,7 @@ protected void clearQryCache(IgniteEx ign) { * @throws IgniteCheckedException In case of errors. */ protected void collectStatistics(StatisticsKey key) throws IgniteCheckedException { - executeSql(String.format("ANALYZE %s.%s", key.schema(), key.obj())); + executeSql(grid(0), String.format("ANALYZE %s.%s", key.schema(), key.obj())); assertTrue( GridTestUtils.waitForCondition( @@ -465,12 +466,12 @@ protected void collectStatistics(StatisticsKey key) throws IgniteCheckedExceptio * @param key Statistics key to collect statistics for. */ protected void dropStatistics(StatisticsKey key) { - executeSql(String.format("DROP STATISTICS %s.%s", key.schema(), key.obj())); + executeSql(grid(0), String.format("DROP STATISTICS %s.%s", key.schema(), key.obj())); } /** */ protected QueryChecker assertQuerySrv(String qry) { - return new QueryChecker(qry) { + return new QueryChecker(qry, tx, sqlTxMode) { @Override protected QueryEngine getEngine() { return Commons.lookupComponent(srv.context(), QueryEngine.class); } @@ -484,23 +485,28 @@ protected QueryChecker assertQuerySrv(String qry) { * @param count Rows count. * @return Populated cache. */ - protected IgniteCache createAndPopulateAllTypesTable(int start, int count) { - IgniteCache all_types = grid(0).getOrCreateCache(new CacheConfiguration() - .setName("all_types") - .setSqlSchema("PUBLIC") - .setQueryEntities(F.asList(new QueryEntity(Integer.class, AllTypes.class).setTableName("all_types"))) - .setBackups(2) - ); + protected IgniteCache populateAllTypesTable(int start, int count) { + IgniteCache all_types = grid(0).cache("all_types"); for (int i = start; i < start + count; i++) { boolean null_values = (i & 3) == 1; - all_types.put(i, new AllTypes(i, null_values)); + put(grid(0), all_types, i, new AllTypes(i, null_values)); } return all_types; } + /** */ + private void createAllTypesTable() { + grid(0).getOrCreateCache(this.cacheConfiguration() + .setName("all_types") + .setSqlSchema("PUBLIC") + .setQueryEntities(F.asList(new QueryEntity(Integer.class, AllTypes.class).setTableName("all_types"))) + .setBackups(2) + ); + } + /** * Test class with fields of all types. */ diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SetOpIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SetOpIntegrationTest.java index 59d0620eed2fb..718937962e0d1 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SetOpIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SetOpIntegrationTest.java @@ -19,8 +19,6 @@ import java.util.Arrays; import java.util.List; - -import javax.cache.Cache; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteDataStreamer; import org.apache.ignite.cache.CacheMode; @@ -33,41 +31,47 @@ /** * Integration test for set op (EXCEPT, INTERSECT). */ -public class SetOpIntegrationTest extends AbstractBasicIntegrationTest { +public class SetOpIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ - private void populateTables() throws InterruptedException { - IgniteCache emp1 = client.getOrCreateCache(new CacheConfiguration("emp1") + private void createTables() { + client.getOrCreateCache(this.cacheConfiguration() + .setName("emp1") .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class).setTableName("emp1"))) .setBackups(2) ); - IgniteCache emp2 = client.getOrCreateCache(new CacheConfiguration("emp2") + client.getOrCreateCache(this.cacheConfiguration() + .setName("emp2") .setSqlSchema("PUBLIC") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class).setTableName("emp2"))) .setBackups(1) ); + } - emp1.put(1, new Employer("Igor", 10d)); - emp1.put(2, new Employer("Igor", 11d)); - emp1.put(3, new Employer("Igor", 12d)); - emp1.put(4, new Employer("Igor1", 13d)); - emp1.put(5, new Employer("Igor1", 13d)); - emp1.put(6, new Employer("Igor1", 13d)); - emp1.put(7, new Employer("Roman", 14d)); - - emp2.put(1, new Employer("Roman", 10d)); - emp2.put(2, new Employer("Roman", 11d)); - emp2.put(3, new Employer("Roman", 12d)); - emp2.put(4, new Employer("Roman", 13d)); - emp2.put(5, new Employer("Igor1", 13d)); - emp2.put(6, new Employer("Igor1", 13d)); + /** */ + private void populateEmp1(IgniteCache emp1) { + put(client, emp1, 1, new Employer("Igor", 10d)); + put(client, emp1, 2, new Employer("Igor", 11d)); + put(client, emp1, 3, new Employer("Igor", 12d)); + put(client, emp1, 4, new Employer("Igor1", 13d)); + put(client, emp1, 5, new Employer("Igor1", 13d)); + put(client, emp1, 6, new Employer("Igor1", 13d)); + put(client, emp1, 7, new Employer("Roman", 14d)); + } - awaitPartitionMapExchange(true, true, null); + /** */ + private void populateEmp2(IgniteCache emp2) { + put(client, emp2, 1, new Employer("Roman", 10d)); + put(client, emp2, 2, new Employer("Roman", 11d)); + put(client, emp2, 3, new Employer("Roman", 12d)); + put(client, emp2, 4, new Employer("Roman", 13d)); + put(client, emp2, 5, new Employer("Igor1", 13d)); + put(client, emp2, 6, new Employer("Igor1", 13d)); } - /** Copy cache with it's content to new replicated cache. */ - private void copyCacheAsReplicated(String cacheName) throws InterruptedException { + /** */ + private void createCacheCopyAsReplicated(String cacheName) throws InterruptedException { IgniteCache cache = client.cache(cacheName); CacheConfiguration ccfg = new CacheConfiguration( @@ -77,10 +81,7 @@ private void copyCacheAsReplicated(String cacheName) throws InterruptedException ccfg.setCacheMode(CacheMode.REPLICATED); ccfg.getQueryEntities().forEach(qe -> qe.setTableName(qe.getTableName() + "_repl")); - IgniteCache replCache = client.getOrCreateCache(ccfg); - - for (Cache.Entry entry : cache) - replCache.put(entry.getKey(), entry.getValue()); + client.getOrCreateCache(ccfg); awaitPartitionMapExchange(true, true, null); } @@ -88,7 +89,9 @@ private void copyCacheAsReplicated(String cacheName) throws InterruptedException /** */ @Test public void testExcept() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name FROM emp1 EXCEPT SELECT name FROM emp2"); @@ -99,10 +102,14 @@ public void testExcept() throws Exception { /** */ @Test public void testExceptFromEmpty() throws Exception { - populateTables(); + createTables(); + createCacheCopyAsReplicated("emp1"); + createCacheCopyAsReplicated("emp2"); - copyCacheAsReplicated("emp1"); - copyCacheAsReplicated("emp2"); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); + populateEmp2(client.cache("emp2Replicated")); List> rows = executeSql("SELECT name FROM emp1 WHERE salary < 0 EXCEPT SELECT name FROM emp2"); @@ -116,7 +123,9 @@ public void testExceptFromEmpty() throws Exception { /** */ @Test public void testExceptSeveralColumns() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name, salary FROM emp1 EXCEPT SELECT name, salary FROM emp2"); @@ -128,7 +137,9 @@ public void testExceptSeveralColumns() throws Exception { /** */ @Test public void testExceptAll() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name FROM emp1 EXCEPT ALL SELECT name FROM emp2"); @@ -140,7 +151,9 @@ public void testExceptAll() throws Exception { /** */ @Test public void testExceptNested() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name FROM emp1 EXCEPT (SELECT name FROM emp1 EXCEPT SELECT name FROM emp2)"); @@ -153,8 +166,12 @@ public void testExceptNested() throws Exception { /** */ @Test public void testExceptReplicatedWithPartitioned() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); + createTables(); + createCacheCopyAsReplicated("emp1"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl EXCEPT ALL SELECT name FROM emp2"); @@ -166,9 +183,14 @@ public void testExceptReplicatedWithPartitioned() throws Exception { /** */ @Test public void testExceptReplicated() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); - copyCacheAsReplicated("emp2"); + createTables(); + createCacheCopyAsReplicated("emp1"); + createCacheCopyAsReplicated("emp2"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); + populateEmp2(client.cache("emp2Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl EXCEPT ALL SELECT name FROM emp2_repl"); @@ -180,8 +202,12 @@ public void testExceptReplicated() throws Exception { /** */ @Test public void testExceptMerge() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); + createTables(); + createCacheCopyAsReplicated("emp1"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl EXCEPT ALL SELECT name FROM emp2 EXCEPT ALL " + "SELECT name FROM emp1 WHERE salary < 11"); @@ -194,20 +220,20 @@ public void testExceptMerge() throws Exception { /** */ @Test public void testSetOpBigBatch() throws Exception { - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("cache1") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Integer.class).setTableName("table1"))) .setBackups(2) ); - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("cache2") .setQueryEntities(F.asList(new QueryEntity(Integer.class, Integer.class).setTableName("table2"))) .setBackups(1) ); - copyCacheAsReplicated("cache1"); - copyCacheAsReplicated("cache2"); + createCacheCopyAsReplicated("cache1"); + createCacheCopyAsReplicated("cache2"); try (IgniteDataStreamer ds1 = client.dataStreamer("cache1"); IgniteDataStreamer ds2 = client.dataStreamer("cache2"); @@ -331,7 +357,9 @@ public void testSetOpBigBatch() throws Exception { /** */ @Test public void testIntersect() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name FROM emp1 INTERSECT SELECT name FROM emp2"); @@ -343,7 +371,9 @@ public void testIntersect() throws Exception { /** */ @Test public void testInstersectAll() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name FROM emp1 INTERSECT ALL SELECT name FROM emp2"); @@ -355,10 +385,15 @@ public void testInstersectAll() throws Exception { /** */ @Test public void testIntersectEmpty() throws Exception { - populateTables(); + createTables(); + createCacheCopyAsReplicated("emp1"); + createCacheCopyAsReplicated("emp2"); - copyCacheAsReplicated("emp1"); - copyCacheAsReplicated("emp2"); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + + populateEmp1(client.cache("emp1Replicated")); + populateEmp2(client.cache("emp2Replicated")); List> rows = executeSql("SELECT name FROM emp1 WHERE salary < 0 INTERSECT SELECT name FROM emp2"); @@ -372,8 +407,12 @@ public void testIntersectEmpty() throws Exception { /** */ @Test public void testIntersectMerge() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); + createTables(); + createCacheCopyAsReplicated("emp1"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl INTERSECT ALL SELECT name FROM emp2 INTERSECT ALL " + "SELECT name FROM emp1 WHERE salary < 14"); @@ -385,9 +424,14 @@ public void testIntersectMerge() throws Exception { /** */ @Test public void testIntersectReplicated() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); - copyCacheAsReplicated("emp2"); + createTables(); + createCacheCopyAsReplicated("emp1"); + createCacheCopyAsReplicated("emp2"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); + populateEmp2(client.cache("emp2Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl INTERSECT ALL SELECT name FROM emp2_repl"); @@ -399,8 +443,12 @@ public void testIntersectReplicated() throws Exception { /** */ @Test public void testIntersectReplicatedWithPartitioned() throws Exception { - populateTables(); - copyCacheAsReplicated("emp1"); + createTables(); + createCacheCopyAsReplicated("emp1"); + + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); + populateEmp1(client.cache("emp1Replicated")); List> rows = executeSql("SELECT name FROM emp1_repl INTERSECT ALL SELECT name FROM emp2"); @@ -412,7 +460,9 @@ public void testIntersectReplicatedWithPartitioned() throws Exception { /** */ @Test public void testIntersectSeveralColumns() throws Exception { - populateTables(); + createTables(); + populateEmp1(client.cache("emp1")); + populateEmp2(client.cache("emp2")); List> rows = executeSql("SELECT name, salary FROM emp1 INTERSECT ALL SELECT name, salary FROM emp2"); @@ -424,8 +474,8 @@ public void testIntersectSeveralColumns() throws Exception { @Test public void testSetOpColocated() { executeSql("CREATE TABLE emp(empid INTEGER, deptid INTEGER, name VARCHAR, PRIMARY KEY(empid, deptid)) " + - "WITH AFFINITY_KEY=deptid"); - executeSql("CREATE TABLE dept(deptid INTEGER, name VARCHAR, PRIMARY KEY(deptid))"); + "WITH AFFINITY_KEY=deptid," + atomicity()); + executeSql("CREATE TABLE dept(deptid INTEGER, name VARCHAR, PRIMARY KEY(deptid)) WITH " + atomicity()); executeSql("INSERT INTO emp VALUES (0, 0, 'test0'), (1, 0, 'test1'), (2, 1, 'test2')"); executeSql("INSERT INTO dept VALUES (0, 'test0'), (1, 'test1'), (2, 'test2')"); @@ -459,7 +509,7 @@ public void testSetOpColocated() { */ @Test public void testSetOpRewindability() { - executeSql("CREATE TABLE test(i INTEGER)"); + executeSql("CREATE TABLE test(i INTEGER) WITH " + atomicity()); executeSql("INSERT INTO test VALUES (1), (2)"); assertQuery("SELECT (SELECT i FROM test EXCEPT SELECT test.i) FROM test") @@ -503,21 +553,24 @@ public void testNumbersCastInExcept() throws Exception { private void doTestNumbersCastInSetOp(String op, Integer... expected) throws InterruptedException { List types = F.asList("TINYINT", "SMALLINT", "INTEGER", "REAL", "FLOAT", "BIGINT", "DOUBLE", "DECIMAL"); - sql(client, "CREATE TABLE t0(id INT PRIMARY KEY, val INTEGER) WITH \"affinity_key=id\""); + sql("CREATE TABLE t0(id INT PRIMARY KEY, val INTEGER) WITH \"affinity_key=id," + atomicity() + "\""); try { - sql(client, "INSERT INTO t0 VALUES (1, 30), (2, 20), (3, 30), (4, 40), (5, 50), (6, 50), (7, null)"); - - for (String tblOpts : Arrays.asList("", " WITH \"template=replicated\"", " WITH \"affinity_key=aff\"")) { + for (String tblOpts : Arrays.asList("", "template=replicated", " affinity_key=aff")) { for (String t2 : types) { - sql(client, "CREATE TABLE t1(id INT, aff INT, val " + t2 + ", PRIMARY KEY(id, aff))" + tblOpts); + sql("CREATE TABLE t1(id INT, aff INT, val " + t2 + ", PRIMARY KEY(id, aff)) " + + "WITH " + tblOpts + (tblOpts.isEmpty() ? "" : ",") + atomicity()); - sql(client, "INSERT INTO t1 VALUES (1, 1, 10), (2, 1, 20), (3, 1, 33), (4, 2, 44), (5, 2, 50), " + + sql("INSERT INTO t0 VALUES (1, 30), (2, 20), (3, 30), (4, 40), (5, 50), (6, 50), (7, null)"); + + sql("INSERT INTO t1 VALUES (1, 1, 10), (2, 1, 20), (3, 1, 33), (4, 2, 44), (5, 2, 50), " + "(6, 3, 50), (7, 3, null)"); - List> res = sql(client, "SELECT val from t0 " + op + " select val from t1 ORDER BY 1 NULLS LAST"); + List> res = sql("SELECT val from t0 " + op + " select val from t1 ORDER BY 1 NULLS LAST"); + + clearTransaction(); - sql(client, "DROP TABLE t1"); + sql("DROP TABLE t1"); assertEquals(expected.length, res.size()); @@ -526,11 +579,17 @@ private void doTestNumbersCastInSetOp(String op, Integer... expected) throws Int assertEquals(expected[i], res.get(i).get(0) == null ? null : ((Number)res.get(i).get(0)).intValue()); } + + sql("DELETE FROM t0"); + + clearTransaction(); } } } finally { - sql(client, "DROP TABLE t0"); + clearTransaction(); + + sql("DROP TABLE t0"); awaitPartitionMapExchange(); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SortAggregateIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SortAggregateIntegrationTest.java index 569d363b29bb1..0fd9530784e8f 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SortAggregateIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/SortAggregateIntegrationTest.java @@ -27,14 +27,13 @@ import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.QueryIndexType; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.junit.Test; /** * Sort aggregate integration test. */ -public class SortAggregateIntegrationTest extends AbstractBasicIntegrationTest { +public class SortAggregateIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** */ public static final int ROWS = 103; @@ -45,6 +44,8 @@ public class SortAggregateIntegrationTest extends AbstractBasicIntegrationTest { /** {@inheritDoc} */ @Override protected void afterTest() { + clearTransaction(); + for (String cacheName : client.cacheNames()) client.cache(cacheName).clear(); } @@ -77,7 +78,8 @@ public class SortAggregateIntegrationTest extends AbstractBasicIntegrationTest { return super.getConfiguration(igniteInstanceName) .setCacheConfiguration( - new CacheConfiguration<>(part.getTableName()) + cacheConfiguration() + .setName(part.getTableName()) .setAffinity(new RendezvousAffinityFunction(false, 8)) .setCacheMode(CacheMode.PARTITIONED) .setQueryEntities(Arrays.asList(tbl1, part)) @@ -88,7 +90,7 @@ public class SortAggregateIntegrationTest extends AbstractBasicIntegrationTest { /** */ @Test public void mapReduceAggregate() throws InterruptedException { - fillCacheTest(grid(0).cache("TEST"), ROWS); + fillCacheTest(client.cache("TEST"), ROWS); List> cursors = executeSql("SELECT /*+ DISABLE_RULE('HashAggregateConverterRule') */" + "SUM(val0), SUM(val1), grp0 FROM TEST " + @@ -110,7 +112,7 @@ public void mapReduceAggregate() throws InterruptedException { /** */ @Test public void correctCollationsOnMapReduceSortAgg() throws InterruptedException { - fillCacheTbl1(grid(0).cache("TEST"), ROWS); + fillCacheTbl1(client.cache("TEST"), ROWS); List> cursors = executeSql("SELECT PK FROM TBL1 WHERE col0 IN (SELECT col0 FROM TBL1)"); @@ -123,7 +125,7 @@ public void correctCollationsOnMapReduceSortAgg() throws InterruptedException { */ private void fillCacheTbl1(IgniteCache c, int rows) throws InterruptedException { for (int i = 0; i < rows; ++i) - c.put(i, new TestValTbl1(i)); + put(client, c, i, new TestValTbl1(i)); awaitPartitionMapExchange(); } @@ -134,7 +136,7 @@ private void fillCacheTbl1(IgniteCache c, int rows) throws InterruptedException */ private void fillCacheTest(IgniteCache c, int rows) throws InterruptedException { for (int i = 0; i < rows; ++i) - c.put(i, new TestValTest(i)); + put(client, c, i, new TestValTest(i)); awaitPartitionMapExchange(); } diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java index 0e1e1435c83e8..fee2099342281 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/TableDmlIntegrationTest.java @@ -34,29 +34,24 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryObjectBuilder; -import org.apache.ignite.cache.query.FieldsQueryCursor; -import org.apache.ignite.cache.query.SqlFieldsQuery; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.query.IgniteSQLException; -import org.apache.ignite.internal.processors.query.QueryEngine; import org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessorTest; import org.apache.ignite.internal.processors.query.calcite.QueryChecker; -import org.apache.ignite.internal.processors.query.calcite.util.Commons; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Test; /** */ -public class TableDmlIntegrationTest extends AbstractBasicIntegrationTest { +public class TableDmlIntegrationTest extends AbstractBasicIntegrationTransactionalTest { /** * Test verifies that already inserted by the current query data * is not processed by this query again. */ @Test public void testInsertAsSelect() { - executeSql("CREATE TABLE test (epoch_cur int, epoch_copied int)"); + executeSql("CREATE TABLE test (epoch_cur int, epoch_copied int) WITH " + atomicity()); executeSql("INSERT INTO test VALUES (0, 0)"); final String insertAsSelectSql = "INSERT INTO test SELECT ?, epoch_cur FROM test"; @@ -76,7 +71,8 @@ public void testInsertAsSelect() { */ @Test public void testInsertAsSelectWithConcurrentDataModification() throws IgniteCheckedException { - executeSql("CREATE TABLE test (id int primary key, val int) with cache_name=\"test\", value_type=\"my_type\""); + executeSql("CREATE TABLE test (id int primary key, val int) " + + "with cache_name=\"test\", value_type=\"my_type\", " + atomicity()); IgniteCache cache = grid(0).cache("test").withKeepBinary(); BinaryObjectBuilder builder = grid(0).binary().builder("my_type"); @@ -110,10 +106,9 @@ public void testInsertAsSelectWithConcurrentDataModification() throws IgniteChec */ @Test public void testUpdate() { - executeSql("CREATE TABLE test (val integer)"); + executeSql("CREATE TABLE test (val integer) with " + atomicity()); - client.context().query().querySqlFields( - new SqlFieldsQuery("CREATE INDEX test_val_idx ON test (val)").setSchema("PUBLIC"), false).getAll(); + executeSql("CREATE INDEX test_val_idx ON test (val)"); for (int i = 1; i <= 4096; i++) executeSql("INSERT INTO test VALUES (?)", i); @@ -144,21 +139,14 @@ public void testUpdate() { /** */ @Test public void testInsertPrimitiveKey() { - grid(1).getOrCreateCache(new CacheConfiguration() + grid(1).getOrCreateCache(this.cacheConfiguration() .setName("developer") .setSqlSchema("PUBLIC") .setIndexedTypes(Integer.class, CalciteQueryProcessorTest.Developer.class) .setBackups(2) ); - QueryEngine engine = Commons.lookupComponent(grid(1).context(), QueryEngine.class); - - List>> qry = engine.query(null, "PUBLIC", - "INSERT INTO DEVELOPER(_key, name, projectId) VALUES (?, ?, ?)", 0, "Igor", 1); - - assertEquals(1, qry.size()); - - List> rows = qry.get(0).getAll(); + List> rows = executeSql("INSERT INTO DEVELOPER(_key, name, projectId) VALUES (?, ?, ?)", 0, "Igor", 1); assertEquals(1, rows.size()); @@ -168,11 +156,11 @@ public void testInsertPrimitiveKey() { assertEqualsCollections(F.asList(1L), row); - qry = engine.query(null, "PUBLIC", "select _key, * from DEVELOPER"); + rows = executeSql("select _key, * from DEVELOPER"); - assertEquals(1, qry.size()); + assertEquals(1, rows.size()); - row = F.first(qry.get(0).getAll()); + row = F.first(rows); assertNotNull(row); @@ -182,7 +170,7 @@ public void testInsertPrimitiveKey() { /** */ @Test public void testInsertUpdateDeleteNonPrimitiveKey() throws Exception { - client.getOrCreateCache(new CacheConfiguration() + client.getOrCreateCache(this.cacheConfiguration() .setName("developer") .setSqlSchema("PUBLIC") .setIndexedTypes(CalciteQueryProcessorTest.Key.class, CalciteQueryProcessorTest.Developer.class) @@ -191,63 +179,37 @@ public void testInsertUpdateDeleteNonPrimitiveKey() throws Exception { awaitPartitionMapExchange(true, true, null); - QueryEngine engine = Commons.lookupComponent(grid(1).context(), QueryEngine.class); - - List>> qry = engine.query(null, "PUBLIC", "INSERT INTO DEVELOPER VALUES (?, ?, ?, ?)", 0, 0, "Igor", 1); - - assertEquals(1, qry.size()); - - List row = F.first(qry.get(0).getAll()); + List row = F.first(executeSql("INSERT INTO DEVELOPER VALUES (?, ?, ?, ?)", 0, 0, "Igor", 1)); assertNotNull(row); assertEqualsCollections(F.asList(1L), row); - qry = engine.query(null, "PUBLIC", "select * from DEVELOPER"); - - assertEquals(1, qry.size()); - - row = F.first(qry.get(0).getAll()); + row = F.first(executeSql("select * from DEVELOPER")); assertNotNull(row); assertEqualsCollections(F.asList(0, 0, "Igor", 1), row); - qry = engine.query(null, "PUBLIC", "UPDATE DEVELOPER d SET name = name || 'Roman' WHERE id = ?", 0); - - assertEquals(1, qry.size()); - - row = F.first(qry.get(0).getAll()); + row = F.first(executeSql("UPDATE DEVELOPER d SET name = name || 'Roman' WHERE id = ?", 0)); assertNotNull(row); assertEqualsCollections(F.asList(1L), row); - qry = engine.query(null, "PUBLIC", "select * from DEVELOPER"); - - assertEquals(1, qry.size()); - - row = F.first(qry.get(0).getAll()); + row = F.first(executeSql("select * from DEVELOPER")); assertNotNull(row); assertEqualsCollections(F.asList(0, 0, "IgorRoman", 1), row); - qry = engine.query(null, "PUBLIC", "DELETE FROM DEVELOPER WHERE id = ?", 0); - - assertEquals(1, qry.size()); - - row = F.first(qry.get(0).getAll()); + row = F.first(executeSql("DELETE FROM DEVELOPER WHERE id = ?", 0)); assertNotNull(row); assertEqualsCollections(F.asList(1L), row); - qry = engine.query(null, "PUBLIC", "select * from DEVELOPER"); - - assertEquals(1, qry.size()); - - row = F.first(qry.get(0).getAll()); + row = F.first(executeSql("select * from DEVELOPER")); assertNull(row); } @@ -257,7 +219,7 @@ public void testInsertUpdateDeleteNonPrimitiveKey() throws Exception { */ @Test public void testInsertUpdateDeleteComplexKey() { - executeSql("CREATE TABLE t(id INT, val VARCHAR, val2 VARCHAR, PRIMARY KEY(id, val))"); + executeSql("CREATE TABLE t(id INT, val VARCHAR, val2 VARCHAR, PRIMARY KEY(id, val)) WITH " + atomicity()); executeSql("INSERT INTO t(id, val, val2) VALUES (1, 'a', 'b')"); assertQuery("SELECT * FROM t").returns(1, "a", "b").check(); @@ -279,11 +241,12 @@ public void testInsertUpdateDeleteComplexKey() { */ @Test public void testMerge() { - executeSql("CREATE TABLE test1 (a int, b varchar, c varchar)"); + executeSql("CREATE TABLE test1 (a int, b varchar, c varchar) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int, b varchar) WITH " + atomicity()); + executeSql("INSERT INTO test1 VALUES (0, 'a', '0')"); executeSql("INSERT INTO test1 VALUES (1, 'b', '1')"); - executeSql("CREATE TABLE test2 (a int, b varchar)"); executeSql("INSERT INTO test2 VALUES (0, '0')"); executeSql("INSERT INTO test2 VALUES (2, '2')"); @@ -305,11 +268,12 @@ public void testMerge() { */ @Test public void testMergeWhenMatched() { - executeSql("CREATE TABLE test1 (a int, b varchar, c varchar)"); + executeSql("CREATE TABLE test1 (a int, b varchar, c varchar) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int, b varchar) WITH " + atomicity()); + executeSql("INSERT INTO test1 VALUES (0, 'a', '0')"); executeSql("INSERT INTO test1 VALUES (1, 'b', '1')"); - executeSql("CREATE TABLE test2 (a int, b varchar)"); executeSql("INSERT INTO test2 VALUES (0, '0')"); executeSql("INSERT INTO test2 VALUES (2, '2')"); @@ -329,11 +293,12 @@ public void testMergeWhenMatched() { */ @Test public void testMergeWhenNotMatched() { - executeSql("CREATE TABLE test1 (a int, b varchar, c varchar)"); + executeSql("CREATE TABLE test1 (a int, b varchar, c varchar) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int, b varchar) WITH " + atomicity()); + executeSql("INSERT INTO test1 VALUES (0, 'a', '0')"); executeSql("INSERT INTO test1 VALUES (1, 'b', '1')"); - executeSql("CREATE TABLE test2 (a int, b varchar)"); executeSql("INSERT INTO test2 VALUES (0, '0')"); executeSql("INSERT INTO test2 VALUES (2, '2')"); @@ -354,7 +319,7 @@ public void testMergeWhenNotMatched() { */ @Test public void testMergeTableWithItself() { - executeSql("CREATE TABLE test1 (a int, b int, c varchar)"); + executeSql("CREATE TABLE test1 (a int, b int, c varchar) WITH " + atomicity()); executeSql("INSERT INTO test1 VALUES (0, 0, '0')"); String sql = "MERGE INTO test1 dst USING test1 src ON dst.a = src.a + 1 " + @@ -379,12 +344,11 @@ public void testMergeTableWithItself() { */ @Test public void testMergeBatch() { - executeSql("CREATE TABLE test1 (a int)"); + executeSql("CREATE TABLE test1 (a int) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int, b int) WITH " + atomicity()); executeSql("INSERT INTO test1 SELECT x FROM TABLE(SYSTEM_RANGE(0, 9999))"); - executeSql("CREATE TABLE test2 (a int, b int)"); - executeSql("INSERT INTO test2 SELECT x, 0 FROM TABLE(SYSTEM_RANGE(-5000, 4999))"); executeSql("MERGE INTO test2 dst USING test1 src ON dst.a = src.a " + @@ -401,10 +365,10 @@ public void testMergeBatch() { */ @Test public void testMergeAliases() { - executeSql("CREATE TABLE test1 (a int, b int, c varchar)"); - executeSql("INSERT INTO test1 VALUES (0, 0, '0')"); + executeSql("CREATE TABLE test1 (a int, b int, c varchar) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int, d int, e varchar) WITH " + atomicity()); - executeSql("CREATE TABLE test2 (a int, d int, e varchar)"); + executeSql("INSERT INTO test1 VALUES (0, 0, '0')"); // Without aliases, column 'A' in insert statement is not ambiguous. executeSql("MERGE INTO test2 USING test1 ON c = e " + @@ -448,12 +412,12 @@ public void testMergeAliases() { */ @Test public void testMergeKeysConflict() { - executeSql("CREATE TABLE test1 (a int, b int)"); + executeSql("CREATE TABLE test1 (a int, b int) WITH " + atomicity()); + executeSql("CREATE TABLE test2 (a int primary key, b int) WITH " + atomicity()); + executeSql("INSERT INTO test1 VALUES (0, 0)"); executeSql("INSERT INTO test1 VALUES (1, 1)"); - executeSql("CREATE TABLE test2 (a int primary key, b int)"); - assertThrows("MERGE INTO test2 USING test1 ON test1.a = test2.a " + "WHEN MATCHED THEN UPDATE SET b = test1.b + 1 " + "WHEN NOT MATCHED THEN INSERT (a, b) VALUES (0, b)", IgniteSQLException.class, @@ -477,7 +441,7 @@ public void testFailureOnNonExistentTable() { IgniteSQLException.class, "Object 'NON_EXISTENT_TABLE' not found"); - executeSql("CREATE TABLE PERSON(ID INT, PRIMARY KEY(id), NAME VARCHAR)"); + executeSql("CREATE TABLE PERSON(ID INT, PRIMARY KEY(id), NAME VARCHAR) WITH " + atomicity()); assertThrows("" + "MERGE INTO PERSON DST USING NON_EXISTENT_TABLE SRC ON DST.ID = SRC.ID" + @@ -500,7 +464,7 @@ public void testInsertMultipleDefaults() { Stream.of(true, false).forEach(withPk -> { try { sql("CREATE TABLE integers(i INTEGER " + (withPk ? "PRIMARY KEY" : "") + - " , col1 INTEGER DEFAULT 200, col2 INTEGER DEFAULT 300)"); + " , col1 INTEGER DEFAULT 200, col2 INTEGER DEFAULT 300) WITH " + atomicity()); sql("INSERT INTO integers (i) VALUES (0)"); sql("INSERT INTO integers VALUES (1, DEFAULT, DEFAULT)"); @@ -525,6 +489,8 @@ public void testInsertMultipleDefaults() { .check(); } finally { + clearTransaction(); + sql("DROP TABLE IF EXISTS integers"); } }); @@ -577,7 +543,7 @@ public void testInsertDefaultValue() { */ @Test public void testInsertDuplicateKey() { - executeSql("CREATE TABLE test (a int primary key, b int)"); + executeSql("CREATE TABLE test (a int primary key, b int) WITH " + atomicity()); executeSql("INSERT INTO test VALUES (0, 0)"); executeSql("INSERT INTO test VALUES (1, 1)"); @@ -602,8 +568,8 @@ public void testInsertValueOverflow() { long max = (Long)arg.get(1); long min = (Long)arg.get(2); - sql(String.format("CREATE TABLE TEST_SOURCE (ID INT PRIMARY KEY, VAL %s)", type)); - sql(String.format("CREATE TABLE TEST_DEST (ID INT PRIMARY KEY, VAL %s)", type)); + sql(String.format("CREATE TABLE TEST_SOURCE (ID INT PRIMARY KEY, VAL %s) WITH " + atomicity(), type)); + sql(String.format("CREATE TABLE TEST_DEST (ID INT PRIMARY KEY, VAL %s) WITH " + atomicity(), type)); sql("INSERT INTO TEST_SOURCE VALUES (1, 1)"); sql(String.format("INSERT INTO TEST_SOURCE VALUES (2, %d)", max)); @@ -628,6 +594,8 @@ public void testInsertValueOverflow() { IgniteSQLException.class, type + " overflow"); } finally { + clearTransaction(); + sql("DROP TABLE TEST_SOURCE"); sql("DROP TABLE TEST_DEST"); } @@ -637,7 +605,7 @@ public void testInsertValueOverflow() { /** */ private void checkDefaultValue(String sqlType, String sqlVal, Object expectedVal) { try { - executeSql("CREATE TABLE test (dummy INT, val " + sqlType + " DEFAULT " + sqlVal + ")"); + executeSql("CREATE TABLE test (dummy INT, val " + sqlType + " DEFAULT " + sqlVal + ") WITH " + atomicity()); executeSql("INSERT INTO test (dummy) VALUES (0)"); checkQueryResult("SELECT val FROM test", expectedVal); @@ -648,6 +616,8 @@ private void checkDefaultValue(String sqlType, String sqlVal, Object expectedVal checkQueryResult("SELECT val FROM test", expectedVal); } finally { + clearTransaction(); + executeSql("DROP TABLE IF EXISTS test"); } } @@ -669,7 +639,7 @@ private void checkQueryResult(String sql, Object expectedVal) { /** */ private void checkWrongDefault(String sqlType, String sqlVal) { try { - assertThrows("CREATE TABLE test (val " + sqlType + " DEFAULT " + sqlVal + ")", + assertThrows("CREATE TABLE test (val " + sqlType + " DEFAULT " + sqlVal + ") WITH " + atomicity(), IgniteSQLException.class, "Cannot convert literal"); } finally { diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/UserDefinedFunctionsIntegrationTransactionalTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/UserDefinedFunctionsIntegrationTransactionalTest.java new file mode 100644 index 0000000000000..c1d9d1b5438e2 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/integration/UserDefinedFunctionsIntegrationTransactionalTest.java @@ -0,0 +1,46 @@ +/* + * 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.processors.query.calcite.integration; + +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.internal.util.typedef.F; +import org.junit.Test; + +/** + * Integration test for user defined functions. + */ +public class UserDefinedFunctionsIntegrationTransactionalTest extends AbstractBasicIntegrationTransactionalTest { + /** */ + @Test + public void testFunctions() { + // Cache with PUBLIC schema. + IgniteCache emp3 = client.getOrCreateCache(this.cacheConfiguration() + .setName("emp3") + .setSqlFunctionClasses(UserDefinedFunctionsIntegrationTest.OtherFunctionsLibrary.class) + .setSqlSchema("PUBLIC") + .setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class).setTableName("emp3"))) + ); + + put(client, emp3, 1, new Employer("Igor3", 100d)); + put(client, emp3, 2, new Employer("Roman3", 200d)); + + assertQuery("SELECT sq(salary) FROM emp3").returns(10_000d).returns(40_000d).check(); + assertQuery("SELECT echo(name) FROM emp3").returns("Igor3").returns("Roman3").check(); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcThinTransactionalSelfTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcThinTransactionalSelfTest.java new file mode 100644 index 0000000000000..22fb84ba4fedf --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/jdbc/JdbcThinTransactionalSelfTest.java @@ -0,0 +1,295 @@ +/* + * 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.processors.query.calcite.jdbc; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Arrays; +import java.util.List; +import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.IgniteConfigVariationsAbstractTest.TestRunnable; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.junit.Test; + +import static java.sql.Connection.TRANSACTION_NONE; +import static java.sql.Connection.TRANSACTION_READ_COMMITTED; +import static java.sql.Connection.TRANSACTION_READ_UNCOMMITTED; +import static java.sql.Connection.TRANSACTION_REPEATABLE_READ; +import static java.sql.Connection.TRANSACTION_SERIALIZABLE; +import static java.sql.ResultSet.CLOSE_CURSORS_AT_COMMIT; +import static java.sql.ResultSet.CONCUR_READ_ONLY; +import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT; +import static java.sql.ResultSet.TYPE_FORWARD_ONLY; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** */ +public class JdbcThinTransactionalSelfTest extends GridCommonAbstractTest { + /** URL. */ + private static final String URL = "jdbc:ignite:thin://127.0.0.1"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(true); + cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration()); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(); + } + + /** */ + @Test + public void testDatabaseMetadata() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + DatabaseMetaData meta = conn.getMetaData(); + + assertTrue(meta.supportsTransactions()); + assertEquals(TRANSACTION_READ_COMMITTED, meta.getDefaultTransactionIsolation()); + assertTrue(meta.supportsDataManipulationTransactionsOnly()); + + assertTrue(meta.supportsTransactionIsolationLevel(TRANSACTION_NONE)); + assertTrue(meta.supportsTransactionIsolationLevel(TRANSACTION_READ_COMMITTED)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_REPEATABLE_READ)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_SERIALIZABLE)); + assertFalse(meta.supportsTransactionIsolationLevel(TRANSACTION_READ_UNCOMMITTED)); + } + } + + /** */ + @Test + public void testInvalidHoldability() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + List checks = Arrays.asList( + () -> conn.setHoldability(HOLD_CURSORS_OVER_COMMIT), + () -> conn.createStatement(TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT), + () -> conn.prepareStatement("SELECT * FROM T", TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT) + ); + + assertEquals(CLOSE_CURSORS_AT_COMMIT, conn.getHoldability()); + + for (TestRunnable check : checks) { + assertThrows( + null, + () -> { + check.run(); + return null; + }, + SQLException.class, + "Invalid holdability (can't hold cursor over commit)." + ); + } + } + } + + /** */ + @Test + public void testTransactionConcurrencyProperty() throws Exception { + for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) { + String url = URL + "?transactionConcurrency=" + txConcurrency; + + try (Connection conn = DriverManager.getConnection(url)) { + conn.setAutoCommit(false); + + try (ResultSet rs = conn.prepareStatement("SELECT 1").executeQuery()) { + assertEquals(1, F.size(grid().context().cache().context().tm().activeTransactions())); + assertEquals(txConcurrency, F.first(grid().context().cache().context().tm().activeTransactions()).concurrency()); + } + } + } + } + + /** */ + @Test + public void testTransactionIsolation() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + assertEquals(TRANSACTION_READ_COMMITTED, conn.getTransactionIsolation()); + + conn.setTransactionIsolation(TRANSACTION_NONE); + + assertEquals(TRANSACTION_NONE, conn.getTransactionIsolation()); + + for (int invalidIsolation : new int[]{TRANSACTION_READ_UNCOMMITTED, TRANSACTION_REPEATABLE_READ, TRANSACTION_SERIALIZABLE}) { + assertThrows( + null, + () -> { + conn.setTransactionIsolation(invalidIsolation); + return null; + }, + SQLException.class, + "Requested isolation level not supported by the server: " + invalidIsolation + ); + } + } + } + + /** */ + @Test + public void testChangeStreamInsideTransactionThrows() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + conn.setAutoCommit(false); + + conn.prepareStatement("SELECT 1").executeQuery(); + + assertThrows( + null, + () -> { + conn.prepareStatement("SET STREAMING ON").executeUpdate(); + return null; + }, + SQLException.class, + "Can't change stream mode inside transaction" + ); + } + } + + /** */ + @Test + public void testNoTxInNoTxIsolation() throws Exception { + try (Connection conn = DriverManager.getConnection(URL)) { + conn.setTransactionIsolation(TRANSACTION_NONE); + + conn.prepareStatement("SELECT 1").executeQuery(); + + assertNull(GridTestUtils.getFieldValue(conn, "txCtx")); + } + } + + /** */ + @Test + public void testTransactionLabel() throws Exception { + String url = URL + "?transactionLabel=mylabel"; + + try (Connection conn = DriverManager.getConnection(url)) { + conn.setAutoCommit(false); + + try (ResultSet rs = conn.prepareStatement("SELECT 1").executeQuery()) { + assertEquals(1, F.size(grid().context().cache().context().tm().activeTransactions())); + assertEquals("mylabel", F.first(grid().context().cache().context().tm().activeTransactions()).label()); + } + } + } + + /** */ + @Test + public void testTransactionTimeout() throws Exception { + int timeout = 1000; + + String url = URL + "?transactionTimeout=" + timeout; + + try (Connection conn = DriverManager.getConnection(url)) { + conn.setAutoCommit(false); + + ResultSet rs = conn.prepareStatement("SELECT 1").executeQuery(); + + Thread.sleep(3 * timeout); + + assertThrows( + null, + () -> { + rs.close(); + conn.commit(); + return null; + }, + SQLException.class, + "Cache transaction timed out" + ); + } + } + + /** */ + @Test + public void testStatementsClosedOnTxEnd() throws Exception { + for (boolean commit : new boolean[]{true, false}) { + + try (Connection conn = DriverManager.getConnection(URL)) { + conn.setAutoCommit(false); + + PreparedStatement stmt0 = conn.prepareStatement("SELECT 1"); + PreparedStatement stmt1 = conn.prepareStatement("SELECT 1"); + + ResultSet rs0 = stmt0.executeQuery(); + ResultSet rs1 = stmt1.executeQuery(); + + assertFalse(stmt0.isClosed()); + assertFalse(stmt1.isClosed()); + assertFalse(rs0.isClosed()); + assertFalse(rs1.isClosed()); + + if (commit) + conn.commit(); + else + conn.rollback(); + + assertFalse(stmt0.isClosed()); + assertFalse(stmt1.isClosed()); + assertTrue(rs0.isClosed()); + assertTrue(rs1.isClosed()); + + stmt0.close(); + assertTrue(stmt0.isClosed()); + assertFalse(stmt1.isClosed()); + + assertNull(GridTestUtils.getFieldValue(conn, "txCtx")); + } + } + } + + /** */ + @Test + public void testCloseConnectionWithoutCommit() throws Exception { + PreparedStatement stmt0; + PreparedStatement stmt1; + ResultSet rs0; + ResultSet rs1; + + try (Connection conn = DriverManager.getConnection(URL)) { + conn.setAutoCommit(false); + + stmt0 = conn.prepareStatement("SELECT 1"); + stmt1 = conn.prepareStatement("SELECT 1"); + + rs0 = stmt0.executeQuery(); + rs1 = stmt1.executeQuery(); + + assertFalse(stmt0.isClosed()); + assertFalse(stmt1.isClosed()); + assertFalse(rs0.isClosed()); + assertFalse(rs1.isClosed()); + } + + assertTrue(stmt0.isClosed()); + assertTrue(stmt1.isClosed()); + assertTrue(rs0.isClosed()); + assertTrue(rs1.isClosed()); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java index baf2809de5467..ab1c61a2cd31e 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/query/calcite/planner/PlannerTest.java @@ -425,7 +425,8 @@ private Node implementFragment( NoOpMemoryTracker.INSTANCE, NoOpIoTracker.INSTANCE, 0, - Commons.parametersMap(ctx.parameters())); + Commons.parametersMap(ctx.parameters()), + null); return new LogicalRelImplementor<>(ectx, c -> r -> 0, mailboxRegistry, exchangeSvc, new TestFailureProcessor(kernal)).go(fragment.root()); diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsIsolationTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsIsolationTest.java new file mode 100644 index 0000000000000..d19cb47fe3058 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsIsolationTest.java @@ -0,0 +1,1431 @@ +/* + * 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.processors.tx; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ThreadLocalRandom; +import java.util.function.BiConsumer; +import java.util.function.IntConsumer; +import java.util.function.IntFunction; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.LongStream; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.EntryProcessorException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.Ignition; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.CacheMode; +import org.apache.ignite.cache.CacheWriteSynchronizationMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; +import org.apache.ignite.client.ClientCache; +import org.apache.ignite.client.ClientTransaction; +import org.apache.ignite.client.Config; +import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.ClientConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.odbc.ClientMessage; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext; +import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; +import org.apache.ignite.internal.processors.query.QueryUtils; +import org.apache.ignite.internal.util.lang.RunnableX; +import org.apache.ignite.internal.util.nio.GridNioServer; +import org.apache.ignite.internal.util.nio.GridNioSession; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static java.lang.String.format; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl.PROP_PREFIX; +import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.CONN_CTX_META_KEY; +import static org.apache.ignite.internal.processors.tx.SqlTransactionsIsolationTest.ModifyApi.CACHE; +import static org.apache.ignite.internal.processors.tx.SqlTransactionsIsolationTest.ModifyApi.ENTRY_PROCESSOR; +import static org.apache.ignite.internal.processors.tx.SqlTransactionsIsolationTest.ModifyApi.SQL; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; +import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED; +import static org.junit.Assume.assumeFalse; + +/** */ +@RunWith(Parameterized.class) +public class SqlTransactionsIsolationTest extends GridCommonAbstractTest { + /** */ + public static final String USERS = "USERS"; + + /** */ + public static final String DEPARTMENTS = "DEPARTMENTS"; + + /** */ + public static final String TBL = "TBL"; + + /** */ + public enum ExecutorType { + /** */ + SERVER, + + /** */ + CLIENT, + + /** */ + THIN_VIA_CACHE_API, + + /** */ + THIN_VIA_QUERY, + + /** */ + THIN_JDBC + } + + /** */ + public enum ModifyApi { + /** */ + CACHE, + + /** */ + ENTRY_PROCESSOR, + + /** */ + SQL + } + + /** */ + public static final User JOHN = new User(1, 0, "John Connor"); + + /** */ + public static final User SARAH = new User(2, 0, "Sarah Connor"); + + /** */ + public static final User KYLE = new User(3, 0, "Kyle Reese"); + + /** */ + public static final int TX_TIMEOUT = 60_000; + + /** */ + public static final int TX_SIZE = 10; + + /** */ + @Parameterized.Parameter() + public ModifyApi modify; + + /** */ + @Parameterized.Parameter(1) + public ExecutorType type; + + /** */ + @Parameterized.Parameter(2) + public boolean partitionAwareness; + + /** */ + @Parameterized.Parameter(3) + public CacheMode mode; + + /** */ + @Parameterized.Parameter(4) + public int gridCnt; + + /** */ + @Parameterized.Parameter(5) + public int backups; + + /** */ + @Parameterized.Parameter(6) + public boolean commit; + + /** */ + @Parameterized.Parameter(7) + public boolean multi; + + /** */ + @Parameterized.Parameter(8) + public TransactionConcurrency txConcurrency; + + /** */ + private static IgniteEx srv; + + /** */ + private static IgniteEx cli; + + /** */ + private static IgniteClient thinCli; + + /** */ + private static ClientConfiguration thinCliCfg; + + /** */ + private ThreadLocal jdbcThinConn = ThreadLocal.withInitial(() -> { + try { + String addrs = partitionAwareness + ? Ignition.allGrids().stream() + .filter(n -> !n.configuration().isClientMode()) + .map(n -> "127.0.0.1:" + ((IgniteEx)n).context().clientListener().port()) + .collect(Collectors.joining(",")) + : "127.0.0.1:10800"; + + return DriverManager.getConnection("jdbc:ignite:thin://" + addrs + "?" + + PROP_PREFIX + "partitionAwareness=" + partitionAwareness + "&" + + PROP_PREFIX + "transactionConcurrency=" + txConcurrency); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + }); + + /** */ + private final TransactionIsolation txIsolation = READ_COMMITTED; + + /** */ + private final Map> partsToKeys = new HashMap<>(); + + /** @return Test parameters. */ + @Parameterized.Parameters( + name = "modify={0},qryExecutor={1},partitionAwareness={2},mode={3},gridCnt={4},backups={5},commit={6},multi={7},txConcurrency={8}") + public static Collection parameters() { + List params = new ArrayList<>(); + + for (CacheMode cacheMode : CacheMode.values()) { + for (int gridCnt : new int[]{1, 3}) { + int[] backups = gridCnt > 1 + ? new int[]{1, gridCnt - 1} + : new int[]{0}; + for (int backup: backups) { + for (ModifyApi modify : ModifyApi.values()) { + for (boolean commit : new boolean[]{false, true}) { + for (boolean mutli : new boolean[] {false, true}) { + for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) { + ExecutorType[] nodeExecTypes = {ExecutorType.SERVER, ExecutorType.CLIENT}; + ExecutorType[] thinExecTypes; + + if (modify == SQL) { + thinExecTypes = new ExecutorType[]{ + ExecutorType.THIN_VIA_CACHE_API, + ExecutorType.THIN_VIA_QUERY, + ExecutorType.THIN_JDBC + }; + } + else { + thinExecTypes = new ExecutorType[]{ + ExecutorType.THIN_VIA_CACHE_API, + ExecutorType.THIN_VIA_QUERY + }; + } + + for (ExecutorType execType : nodeExecTypes) { + params.add(new Object[]{ + modify, + execType, + false, //partitionAwareness + cacheMode, + gridCnt, + backup, + commit, + mutli, + txConcurrency + }); + } + + for (ExecutorType execType : thinExecTypes) { + for (boolean partitionAwareness : new boolean[]{false, true}) { + params.add(new Object[]{modify, + execType, + partitionAwareness, + cacheMode, + gridCnt, + backup, + commit, + mutli, + txConcurrency + }); + } + } + } + } + } + } + } + } + } + + return params; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration()); + cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(true); + + return cfg; + } + + /** */ + private void startAllGrids() throws Exception { + srv = startGrids(gridCnt); + cli = startClientGrid("client"); + } + + /** */ + private void startThinClient() { + thinCliCfg = new ClientConfiguration() + .setAddresses(Config.SERVER) + .setPartitionAwarenessEnabled(partitionAwareness); + thinCli = Ignition.startClient(thinCliCfg); + } + + /** */ + private void createCaches() { + cli.createCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setCacheMode(mode) + .setBackups(backups)); + + cli.createCache(new CacheConfiguration() + .setName(tableName(TBL, mode)) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setBackups(backups) + .setCacheMode(mode) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setSqlSchema(QueryUtils.DFLT_SCHEMA) + .setQueryEntities(Collections.singleton(new QueryEntity() + .setTableName(tableName(TBL, mode)) + .setKeyType(Long.class.getName()) + .setValueType(Long.class.getName())))); + + for (CacheMode mode : CacheMode.values()) { + String users = tableName(USERS, mode); + String deps = tableName(DEPARTMENTS, mode); + + LinkedHashMap userFlds = new LinkedHashMap<>(); + + userFlds.put("id", Integer.class.getName()); + userFlds.put("userId", Integer.class.getName()); + userFlds.put("departmentId", Integer.class.getName()); + userFlds.put("fio", String.class.getName()); + + LinkedHashMap depFlds = new LinkedHashMap<>(); + + depFlds.put("id", Integer.class.getName()); + depFlds.put("name", String.class.getName()); + + cli.createCache(new CacheConfiguration<>() + .setName(users) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setCacheMode(mode) + .setBackups(backups) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setSqlSchema(QueryUtils.DFLT_SCHEMA) + .setQueryEntities(Collections.singleton(new QueryEntity() + .setTableName(users) + .setKeyType(Integer.class.getName()) + .setValueType(User.class.getName()) + .setKeyFieldName("id") + .setFields(userFlds) + .setIndexes(Arrays.asList( + new QueryIndex() + .setName("IDX_FIO_" + users) + .setFieldNames(Collections.singleton("fio"), true).setInlineSize(Character.BYTES * 20), + new QueryIndex() + .setName("IDX_USERID_" + users) + .setFieldNames(Collections.singleton("userId"), true) + ))))); + + cli.createCache(new CacheConfiguration<>() + .setName(deps) + .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) + .setCacheMode(mode) + .setBackups(backups) + .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC) + .setSqlSchema(QueryUtils.DFLT_SCHEMA) + .setQueryEntities(Collections.singleton(new QueryEntity() + .setTableName(deps) + .setKeyType(Integer.class.getName()) + .setValueType(String.class.getName()) + .setKeyFieldName("id") + .setFields(depFlds)))); + } + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + super.afterTestsStopped(); + + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + super.beforeTest(); + + if (gridCnt != Ignition.allGrids().size() - 1) { + stopAllGrids(); + + startAllGrids(); + + startThinClient(); + + createCaches(); + } + + if (partitionAwareness != thinCliCfg.isPartitionAwarenessEnabled()) + startThinClient(); + + boolean recreate = cli.cacheNames().stream() + .map(name -> cli.cache(name).getConfiguration(CacheConfiguration.class)) + .filter(ccfg -> ccfg.getCacheMode() == CacheMode.PARTITIONED) + .anyMatch(ccfg -> backups != ccfg.getBackups()); + + if (recreate) { + cli.cacheNames().forEach(cli::destroyCache); + + createCaches(); + } + + cli.cache(users()).removeAll(); + cli.cache(tbl()).removeAll(); + cli.cache(departments()).removeAll(); + + insert(F.t(1, JOHN)); + + assertEquals(gridCnt + 1, Ignition.allGrids().size()); + assertEquals(partitionAwareness, thinCliCfg.isPartitionAwarenessEnabled()); + assertEquals(mode, cli.cache(tbl()).getConfiguration(CacheConfiguration.class).getCacheMode()); + cli.cacheNames().stream() + .map(name -> cli.cache(name).getConfiguration(CacheConfiguration.class)) + .filter(ccfg -> ccfg.getCacheMode() == CacheMode.PARTITIONED) + .forEach(ccfg -> assertEquals(ccfg.getName(), backups, ccfg.getBackups())); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + for (int i = 0; i < gridCnt + 1; i++) { + IgniteEx srv = i == gridCnt ? cli : grid(i); + + assertTrue(srv.context().cache().context().tm().activeTransactions().isEmpty()); + + GridNioServer nioSrv = GridTestUtils.getFieldValue(srv.context().clientListener(), "srv"); + + for (GridNioSession ses : nioSrv.sessions()) { + Object meta = ses.meta(CONN_CTX_META_KEY); + + if (meta instanceof ClientConnectionContext) + assertTrue(GridTestUtils.>getFieldValue(meta, "txs").isEmpty()); + else if (meta instanceof JdbcConnectionContext) + assertNull(GridTestUtils.getFieldValue(meta, "txCtx")); + else + throw new IllegalStateException("Unknown context"); + } + } + } + + /** */ + @Test + public void testIndexScan() { + delete(1); + + assertUsersSize(0); + + int stepCnt = 7; + int stepSz = 12; + int outOfTxSz = 3; + int inTxSz = 9; + + assert outOfTxSz + inTxSz == stepSz; + + for (int i = 0; i < stepCnt; i++) { + int outOfTxStart = i * stepSz; + + for (int j = 0; j < outOfTxSz; j++) { + int id = outOfTxStart + j; + + insert(F.t(id, new User(id, 0, "User" + j))); // Intentionally repeat FIO to make same indexed keys. + } + } + + assertUsersSize(stepCnt * outOfTxSz); + + checkQueryWithPartitionFilter(); + + insideTx(() -> { + int nullFio = 0; + + for (int i = 0; i < stepCnt; i++) { + int outOfTxStart = i * stepSz; + int inTxStart = i * stepSz + outOfTxSz; + + assertEquals( + outOfTxStart, + sql(format("SELECT MIN(userid) FROM %s WHERE userid >= ?", users()), outOfTxStart).get(0).get(0) + ); + assertEquals( + outOfTxStart + 1, + sql(format("SELECT MIN(userid) FROM %s WHERE userid > ?", users()), outOfTxStart).get(0).get(0) + ); + + for (int j = 0; j < inTxSz; j++) { + final int id = inTxStart + j; + + String fio = ThreadLocalRandom.current().nextBoolean() + ? ("User" + j) // Intentionally repeat FIO to make same indexed keys. + : null; + + if (fio == null) + nullFio++; + + insert(F.t(id, new User(id, 0, fio))); + + if (txConcurrency == TransactionConcurrency.OPTIMISTIC) { + // Concurrent query must not see any transaction data. + runAsync(() -> { + RunnableX check = () -> { + assertUsersSize(stepCnt * outOfTxSz); + + if (type != ExecutorType.THIN_JDBC) + assertNull(select(id, CACHE)); + assertNull(select(id, SQL)); + }; + + insideTx(check, false); + check.run(); + }).get(TX_TIMEOUT); + } + + long expTblSz = (long)(stepCnt * outOfTxSz) + i * inTxSz + j + 1; + + assertUsersSize(expTblSz); + + List> rows = sql(format("SELECT fio FROM %s ORDER BY fio", users())); + + assertEquals(expTblSz, rows.size()); + + ensureSorted(rows, true); + + rows = sql(format("SELECT COUNT(fio) FROM %s", users())); + + assertEquals(expTblSz - nullFio, rows.get(0).get(0)); + + assertEquals( + id, + sql(format("SELECT MIN(userid) FROM %s WHERE userid BETWEEN ? AND ?", users()), id, 500).get(0).get(0) + ); + } + + checkQueryWithPartitionFilter(); + + List> res = sql( + format("SELECT id FROM %s WHERE userId >= ? AND userId <= ? ORDER BY id LIMIT ?", users()), + i * stepSz, + (i + 1) * stepSz, + stepSz + ); + + assertEquals(stepSz, res.size()); + + for (int j = 0; j < stepSz; j++) + assertEquals(outOfTxStart + j, res.get(j).get(0)); + + int offset = 3; + + res = sql( + format("SELECT id FROM %s WHERE userId >= ? AND userId <= ? ORDER BY id LIMIT ? OFFSET ?", users()), + i * stepSz, + (i + 1) * stepSz, + stepSz - offset, + offset + ); + + assertEquals(stepSz - offset, res.size()); + + for (int j = 0; j < stepSz - offset; j++) + assertEquals(outOfTxStart + j + offset, res.get(j).get(0)); + } + + for (int i = 0; i < stepCnt; i++) { + int start = i * stepSz; + + for (int j = 0; j < outOfTxSz; j++) { + int id = start + j; + + delete(id); + + long expTblSz = (stepCnt * stepSz) - (i * outOfTxSz + j + 1); + + assertUsersSize(expTblSz); + + List> rows = sql(format("SELECT fio FROM %s ORDER BY fio DESC", users())); + + assertEquals(expTblSz, rows.size()); + + ensureSorted(rows, false); + } + } + + checkQueryWithPartitionFilter(); + }, true); + + assertUsersSize(inTxSz * stepCnt); + } + + /** */ + @Test + public void testJoin() { + partsToKeys.clear(); + + sql(format("DELETE FROM %s", users())); + sql(format("DELETE FROM %s", departments())); + + assertUsersSize(0); + assertTableSize(0, departments()); + + int depCnt = 5; + int userCnt = 5; + + Map> data = new HashMap<>(); + + IntFunction uname = i -> "User " + i + 1; + + IntConsumer insertDepartment = i -> { + data.put(i + 1, new ArrayList<>()); + + sql(format("INSERT INTO %s(ID, NAME) VALUES(?, ?)", departments()), i + 1, (i + 1) + " department"); + }; + + BiConsumer insertUser = (id, depId) -> { + User user = new User(id, depId, uname.apply(depId)); + + insert(F.t(id, user)); + + data.get(depId).add(user); + }; + + for (int i = 0; i < depCnt; i++) + insertDepartment.accept(i); + + for (int i = 0; i < depCnt; i++) { + for (int j = 0; j < userCnt; j++) + insertUser.accept(i * depCnt + j, j + 1); + } + + insertDepartment.accept(depCnt); + + checkJoin(data, uname); + + insideTx(() -> { + for (int i = depCnt + 1; i < depCnt * 2; i++) + insertDepartment.accept(i); + + for (int i = 0; i < depCnt * 2; i++) { + for (int j = 0; j < userCnt; j++) { + if (i >= depCnt) + insertUser.accept(i * depCnt + j, i + 1); + + checkJoin(data, uname); + } + } + }, false); + } + + /** */ + private void checkJoin(Map> data, IntFunction uname) { + List> res = sql( + format("SELECT d.id, u.id, u.fio FROM %s d JOIN %s u ON d.id = u.departmentId", departments(), users()) + ); + + Map> data0 = new HashMap<>(); + + data.forEach((key, value) -> data0.put(key, new ArrayList<>(value))); + + assertEquals(data0.values().stream().mapToInt(List::size).sum(), res.size()); + + for (List row : res) { + List users = data0.get(row.get(0)); + + assertNotNull(users); + + assertTrue(users.removeIf(u -> { + if (u.userId != (Integer)row.get(1)) + return false; + + assertEquals(u.fio, row.get(2)); + + return true; + })); + } + + data0.values().forEach(l -> assertTrue(l.isEmpty())); + + res = sql( + format("SELECT d.id, COUNT(*) FROM %s d JOIN %s u ON d.id = u.departmentId GROUP BY d.id", departments(), users()) + ); + + assertEquals((int)data.values().stream().filter(((Predicate>)List::isEmpty).negate()).count(), res.size()); + + for (List row : res) + assertEquals((long)data.get(row.get(0)).size(), row.get(1)); + + res = sql( + format("SELECT d.id, COUNT(*) FROM %s d LEFT JOIN %s u ON d.id = u.departmentId GROUP BY d.id", departments(), users()) + ); + + assertEquals(data.size(), res.size()); + + for (List row : res) { + long size = data.get(row.get(0)).size(); + + assertEquals(size == 0 ? 1 : size, row.get(1)); + } + + res = sql( + format("SELECT d.id, u.fio FROM %s d JOIN %s u ON d.id = u.departmentId", departments(), users()) + ); + + assertEquals(data.values().stream().mapToInt(List::size).sum(), res.size()); + + for (List row : res) + assertEquals(uname.apply((Integer)row.get(0)), row.get(1)); + } + + /** */ + @Test + public void testInsert() { + ensureModeSupported(); + + Runnable checkBefore = () -> { + for (int i = 4; i <= (multi ? 6 : 4); i++) { + if (type != ExecutorType.THIN_JDBC) + assertNull(CACHE.name(), select(i, CACHE)); + assertNull(SQL.name(), select(i, SQL)); + } + }; + + Runnable checkAfter = () -> { + for (int i = 4; i <= (multi ? 6 : 4); i++) { + if (type != ExecutorType.THIN_JDBC) + assertEquals(CACHE.name(), JOHN, select(i, CACHE)); + assertEquals(SQL.name(), JOHN, select(i, SQL)); + } + }; + + checkBefore.run(); + + insideTx(() -> { + checkBefore.run(); + + if (multi) + insert(F.t(4, JOHN), F.t(5, JOHN), F.t(6, JOHN)); + else + insert(F.t(4, JOHN)); + + if (modify == SQL) { + assertThrows( + log, + () -> doInsert(false, F.t(4, JOHN)), + IgniteException.class, + "Failed to INSERT some keys because they are already in cache" + ); + } + + checkAfter.run(); + }, commit); + + if (commit) + checkAfter.run(); + else + checkBefore.run(); + } + + /** */ + @Test + public void testUpdate() { + ensureModeSupported(); + + if (multi) + insert(F.t(2, JOHN), F.t(3, JOHN)); + + Runnable checkBefore = () -> { + for (int i = 1; i <= (multi ? 3 : 1); i++) { + if (type != ExecutorType.THIN_JDBC) + assertEquals(JOHN, select(i, CACHE)); + assertEquals(JOHN, select(i, SQL)); + } + }; + + Runnable checkAfter = () -> { + for (int i = 1; i <= (multi ? 3 : 1); i++) { + if (type != ExecutorType.THIN_JDBC) + assertEquals(KYLE, select(i, CACHE)); + assertEquals(KYLE, select(i, SQL)); + } + }; + + checkBefore.run(); + + insideTx(() -> { + checkBefore.run(); + + if (multi) + update(F.t(1, SARAH), F.t(2, SARAH), F.t(3, SARAH)); + else + update(F.t(1, SARAH)); + + for (int i = 1; i <= (multi ? 3 : 1); i++) { + if (type != ExecutorType.THIN_JDBC) + assertEquals(SARAH, select(i, CACHE)); + assertEquals(SARAH, select(i, SQL)); + } + + if (multi) + update(F.t(1, KYLE), F.t(2, KYLE), F.t(3, KYLE)); + else + update(F.t(1, KYLE)); + + checkAfter.run(); + }, commit); + + if (commit) + checkAfter.run(); + else + checkBefore.run(); + } + + /** */ + @Test + public void testDelete() { + ensureModeSupported(); + + if (multi) + insert(F.t(2, JOHN), F.t(3, JOHN)); + + Runnable checkBefore = () -> { + for (int i = 1; i <= (multi ? 3 : 1); i++) { + if (type != ExecutorType.THIN_JDBC) + assertEquals(JOHN, select(i, CACHE)); + assertEquals(JOHN, select(i, SQL)); + } + }; + + Runnable checkAfter = () -> { + for (int i = 1; i <= (multi ? 3 : 1); i++) { + if (type != ExecutorType.THIN_JDBC) + assertNull(select(i, CACHE)); + assertNull(select(i, SQL)); + } + }; + + checkBefore.run(); + + insideTx(() -> { + checkBefore.run(); + + if (multi) + delete(1, 2, 3); + else + delete(1); + + checkAfter.run(); + }, commit); + + if (commit) + checkAfter.run(); + else + checkBefore.run(); + } + + /** */ + @Test + public void testVisibility() { + assumeFalse(type == ExecutorType.THIN_JDBC); + + sql(format("DELETE FROM %s", tbl())); + + assertTableSize(0, tbl()); + + long cnt = 100; + + LongStream.range(1, 1 + cnt).forEach(i -> insideTx(() -> { + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) { + ClientCache thinCache = thinCli.cache(tbl()); + + thinCache.put(i, i + 1); + + assertEquals("Must see transaction related data", (Long)(i + 1), thinCache.get(i)); + } + else { + IgniteCache cache = node().cache(tbl()); + + cache.put(i, i + 1); + + assertEquals("Must see transaction related data", (Long)(i + 1), cache.get(i)); + } + + assertTableSize(i, tbl()); + }, true)); + + assertTableSize(cnt, tbl()); + } + + /** */ + private void insideTx(RunnableX test, boolean commit) { + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) { + try (ClientTransaction tx = thinCli.transactions().txStart(txConcurrency, txIsolation, TX_TIMEOUT)) { + for (int i = 0; i < 3; i++) + thinCli.cache(DEFAULT_CACHE_NAME).put(i, i); + + test.run(); + + if (commit) { + for (int i = 0; i < 3; i++) + thinCli.cache(DEFAULT_CACHE_NAME).remove(i, i); + + tx.commit(); + } + else + tx.rollback(); + } + } + else if (type == ExecutorType.THIN_JDBC) { + try { + jdbcThinConn.get().setAutoCommit(false); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + + try { + test.run(); + } + finally { + try { + if (commit) + jdbcThinConn.get().commit(); + else + jdbcThinConn.get().rollback(); + + jdbcThinConn.get().setAutoCommit(true); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + } + else { + Ignite initiator = node(); + + assertNotNull(initiator); + + try (Transaction tx = initiator.transactions().txStart(txConcurrency, txIsolation, TX_TIMEOUT, TX_SIZE)) { + for (int i = 0; i < 3; i++) + initiator.cache(DEFAULT_CACHE_NAME).put(i, i); + + test.run(); + + if (commit) { + for (int i = 0; i < 3; i++) + initiator.cache(DEFAULT_CACHE_NAME).remove(i, i); + + tx.commit(); + } + else + tx.rollback(); + } + } + } + + /** */ + private User select(Integer id, ModifyApi api) { + if (api == CACHE) + return (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + ? (User)thinCli.cache(users()).get(id) + : (User)node().cache(users()).get(id); + else if (api == SQL) { + List> res = sql(format("SELECT _VAL FROM %s WHERE _KEY = ?", users()), id); + + assertNotNull(res); + + return res.isEmpty() ? null : ((User)res.get(0).get(0)); + } + + fail("Unknown select: " + api); + + return null; + } + + /** */ + private void insert(IgniteBiTuple... entries) { + for (IgniteBiTuple data : entries) { + assertTrue(partsToKeys + .computeIfAbsent(userPartition(data.get1()), k -> new HashSet<>()) + .add(data.get1())); + } + + doInsert(false, entries); + } + + /** */ + private void doInsert(boolean update, IgniteBiTuple... entries) { + if (modify == CACHE) { + if (multi) { + Map data = Arrays.stream(entries).collect(Collectors.toMap(IgniteBiTuple::get1, IgniteBiTuple::get2)); + + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + thinCli.cache(users()).putAll(data); + else + node().cache(users()).putAll(data); + } + else { + for (IgniteBiTuple data : entries) { + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + thinCli.cache(users()).put(data.get1(), data.get2()); + else + node().cache(users()).put(data.get1(), data.get2()); + } + } + } + else if (modify == ENTRY_PROCESSOR) { + if (multi) { + Set keys = Arrays.stream(entries).map(IgniteBiTuple::get1).collect(Collectors.toSet()); + Map data = Arrays.stream(entries).collect(Collectors.toMap(IgniteBiTuple::get1, IgniteBiTuple::get2)); + + if (type == ExecutorType.THIN_VIA_QUERY || type == ExecutorType.THIN_VIA_CACHE_API) + thinCli.cache(users()).invokeAll(keys, new UpdateEntryProcessor<>(update), data); + else + node().cache(users()).invokeAll(keys, new UpdateEntryProcessor<>(update), data); + } + else { + for (IgniteBiTuple data : entries) { + if (type == ExecutorType.THIN_VIA_QUERY || type == ExecutorType.THIN_VIA_CACHE_API) + thinCli.cache(users()).invoke(data.get1(), new UpdateEntryProcessor<>(update), data.get2()); + else + node().cache(users()).invoke(data.get1(), new UpdateEntryProcessor<>(update), data.get2()); + } + } + } + else if (modify == SQL) { + String insert = format("INSERT INTO %s(id, userid, departmentId, fio) VALUES(?, ?, ?, ?)", users()); + + int colCnt = 4; + + if (multi) { + StringBuilder sql = new StringBuilder(); + Object[] params = new Object[entries.length * colCnt]; + + for (int i = 0; i < entries.length; i++) { + IgniteBiTuple data = entries[i]; + + if (i != 0) + sql.append(";"); + + sql.append(insert); + + params[i * colCnt] = data.get1(); + params[i * colCnt + 1] = data.get2().userId; + params[i * colCnt + 2] = data.get2().departmentId; + params[i * colCnt + 3] = data.get2().fio; + } + + sql(sql.toString(), params); + } + else { + for (IgniteBiTuple data : entries) + sql(insert, data.get1(), data.get2().userId, data.get2().departmentId, data.get2().fio); + } + } + else + fail("Unknown insert: " + modify); + } + + /** */ + private void update(IgniteBiTuple...entries) { + for (IgniteBiTuple data : entries) { + int part = userPartition(data.get1()); + + assertTrue(partsToKeys.containsKey(part)); + assertTrue(partsToKeys.get(part).contains(data.get1())); + } + + if (modify == CACHE || modify == ENTRY_PROCESSOR) + doInsert(true, entries); + else if (modify == SQL) { + String update = format("UPDATE %s SET userid = ?, departmentId = ?, fio = ? WHERE id = ?", users()); + + int colCnt = 4; + + if (multi) { + StringBuilder sql = new StringBuilder(); + Object[] params = new Object[entries.length * colCnt]; + + for (int i = 0; i < entries.length; i++) { + IgniteBiTuple data = entries[i]; + + if (i != 0) + sql.append(";"); + + sql.append(update); + + params[i * colCnt] = data.get2().userId; + params[i * colCnt + 1] = data.get2().departmentId; + params[i * colCnt + 2] = data.get2().fio; + params[i * colCnt + 3] = data.get1(); + } + + sql(sql.toString(), params); + } + else { + for (IgniteBiTuple data : entries) + sql(update, data.get2().userId, data.get2().departmentId, data.get2().fio, data.get1()); + } + } + else + fail("Unknown update: " + modify); + } + + /** */ + private void delete(int... keys) { + for (int key : keys) { + int part = userPartition(key); + + assertTrue(partsToKeys.containsKey(part)); + assertTrue(partsToKeys.get(part).remove(key)); + } + + if (modify == CACHE) { + if (multi) { + Set toRemove = Arrays.stream(keys).boxed().collect(Collectors.toSet()); + + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + thinCli.cache(users()).removeAll(toRemove); + else + node().cache(users()).removeAll(toRemove); + } + else { + for (int id : keys) { + if (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + thinCli.cache(users()).remove(id); + else + node().cache(users()).remove(id); + } + } + } + else if (modify == ENTRY_PROCESSOR) { + if (multi) { + Set toRemove = Arrays.stream(keys).boxed().collect(Collectors.toSet()); + + if (type == ExecutorType.THIN_VIA_QUERY || type == ExecutorType.THIN_VIA_CACHE_API) + thinCli.cache(users()).invokeAll(toRemove, new RemoveEntryProcessor<>()); + else + node().cache(users()).invokeAll(toRemove, new RemoveEntryProcessor<>()); + } + else { + for (int id : keys) { + if (type == ExecutorType.THIN_VIA_QUERY || type == ExecutorType.THIN_VIA_CACHE_API) + thinCli.cache(users()).invoke(id, new RemoveEntryProcessor<>()); + else + node().cache(users()).invoke(id, new RemoveEntryProcessor<>()); + } + } + } + else if (modify == SQL) { + String delete = format("DELETE FROM %s WHERE id = ?", users()); + + if (multi) { + StringBuilder sql = new StringBuilder(); + + for (int i = 0; i < keys.length; i++) { + if (i != 0) + sql.append(";"); + + sql.append(delete); + } + + sql(sql.toString(), Arrays.stream(keys).boxed().toArray(Object[]::new)); + } + else { + for (int id : keys) + sql(delete, id); + } + } + else + fail("Unknown delete: " + modify); + } + + /** */ + private void checkQueryWithPartitionFilter() { + // https://issues.apache.org/jira/browse/IGNITE-22993 + if (mode != CacheMode.PARTITIONED) + return; + + // Partitions filter not supported by JDBC. + if (type == ExecutorType.THIN_JDBC) + return; + + for (Map.Entry> partToKeys : partsToKeys.entrySet()) { + assertEquals( + (long)partToKeys.getValue().size(), + sql(format("SELECT COUNT(*) FROM %s", users()), new int[]{partToKeys.getKey()}).get(0).get(0) + ); + + assertEquals( + partToKeys.getValue().size(), + sql(format("SELECT * FROM %s", users()), new int[]{partToKeys.getKey()}).size() + ); + } + } + + /** */ + private void assertUsersSize(long sz) { + assertTableSize(sz, users()); + } + + /** */ + private void assertTableSize(long sz, String tbl) { + assertEquals(sz, sql(format("SELECT COUNT(*) FROM %s", tbl)).get(0).get(0)); + } + + /** */ + private static void ensureSorted(List> rows, boolean asc) { + for (int k = 1; k < rows.size(); k++) { + String fio0 = (String)rows.get(k - 1).get(0); + String fio1 = (String)rows.get(k).get(0); + + int cmp = fio0 == null + ? (fio1 == null ? 0 : -1) + : (fio1 == null ? 1 : fio0.compareTo(fio1)); + + assertTrue(asc ? (cmp <= 0) : (cmp >= 0)); + } + } + + /** */ + public static class User { + /** */ + private final int userId; + + /** */ + private final int departmentId; + + /** */ + private final String fio; + + /** */ + public User(int userId, int departmentId, String fio) { + this.userId = userId; + this.departmentId = departmentId; + this.fio = fio; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + User user = (User)o; + return userId == user.userId && departmentId == user.departmentId && Objects.equals(fio, user.fio); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(userId, departmentId, fio); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(User.class, this); + } + } + + /** */ + public List> sql(String sqlText, Object... args) { + return sql(sqlText, null, args); + } + + /** */ + public List> sql(String sqlText, int[] parts, Object... args) { + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText) + .setArgs(args) + .setTimeout(5, SECONDS); + + if (!F.isEmpty(parts)) + qry.setPartitions(parts); + + if (type == ExecutorType.THIN_VIA_QUERY) + return unwrapBinary(thinCli.query(qry).getAll()); + else if (type == ExecutorType.THIN_VIA_CACHE_API) + return unwrapBinary(thinCli.cache(F.first(thinCli.cacheNames())).query(qry).getAll()); + else if (type == ExecutorType.THIN_JDBC) { + assertTrue("Partition filter not supported", F.isEmpty(parts)); + + try { + PreparedStatement stmt = jdbcThinConn.get().prepareStatement(sqlText); + + if (!F.isEmpty(args)) { + for (int i = 0; i < args.length; i++) + stmt.setObject(i + 1, args[i]); + } + + List> res = new ArrayList<>(); + + if (sqlText.startsWith("SELECT")) { + try (ResultSet rset = stmt.executeQuery()) { + int colCnt = rset.getMetaData().getColumnCount(); + + while (rset.next()) { + List row = new ArrayList<>(); + + res.add(row); + + for (int i = 0; i < colCnt; i++) + row.add(rset.getObject(i + 1)); + } + + return res; + } + } + else + res.add(List.of(stmt.executeUpdate())); + + return res; + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + if (multi) + return node().context().query().querySqlFields(qry, false, false).get(0).getAll(); + else + return node().cache(F.first(cli.cacheNames())).query(qry).getAll(); + } + + /** */ + private IgniteEx node() { + assertTrue(type == ExecutorType.CLIENT || type == ExecutorType.SERVER); + + return type == ExecutorType.CLIENT ? cli : srv; + } + + /** */ + private int userPartition(int id) { + return affinity(cli.cache(users())).partition(id); + } + + /** */ + private String users() { + return tableName(USERS, mode); + } + + /** */ + private String departments() { + return tableName(DEPARTMENTS, mode); + } + + /** */ + private String tbl() { + return tableName(TBL, mode); + } + + /** */ + private static String tableName(String tbl, CacheMode mode) { + return tbl + "_" + mode; + } + + /** */ + private void ensureModeSupported() { + assumeFalse( + "Thin client doesn't support multiple statements for SQL", + multi && modify == SQL && (type == ExecutorType.THIN_VIA_CACHE_API || type == ExecutorType.THIN_VIA_QUERY) + ); + } + + /** */ + private List> unwrapBinary(List> all) { + return all.stream() + .map(row -> row.stream() + .map(col -> col instanceof BinaryObject ? ((BinaryObject)col).deserialize() : col) + .collect(Collectors.toList())) + .collect(Collectors.toList()); + } + + /** */ + public static class RemoveEntryProcessor implements EntryProcessor { + /** {@inheritDoc} */ + @Override public T process(MutableEntry entry, Object... arguments) throws EntryProcessorException { + entry.remove(); + + return null; + } + } + + /** */ + public static class UpdateEntryProcessor implements EntryProcessor { + /** */ + private final boolean update; + + /** */ + public UpdateEntryProcessor() { + this(true); + } + + /** */ + public UpdateEntryProcessor(boolean update) { + this.update = update; + } + + /** {@inheritDoc} */ + @Override public T process(MutableEntry entry, Object... arguments) throws EntryProcessorException { + assertEquals("Expect entry " + (update ? "" : "not") + " exists", update, entry.exists()); + + if (arguments[0] instanceof User) + entry.setValue((V)arguments[0]); + else + entry.setValue((V)((Map)arguments[0]).get((Integer)entry.getKey())); + + return null; + } + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsUnsupportedModesTest.java b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsUnsupportedModesTest.java new file mode 100644 index 0000000000000..78fdfa9cb5f91 --- /dev/null +++ b/modules/calcite/src/test/java/org/apache/ignite/internal/processors/tx/SqlTransactionsUnsupportedModesTest.java @@ -0,0 +1,96 @@ +/* + * 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.processors.tx; + +import java.util.EnumSet; +import java.util.List; +import javax.cache.CacheException; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.calcite.CalciteQueryEngineConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.junit.Test; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** */ +public class SqlTransactionsUnsupportedModesTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration()); + cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(igniteInstanceName.contains("0")); + + return cfg; + } + + /** */ + @Test + public void testUnsupportedTransactionModes() throws Exception { + try (IgniteEx srv = startGrid(0)) { + sql("CREATE TABLE T1(ID BIGINT PRIMARY KEY, NAME VARCHAR(100))"); + + for (TransactionConcurrency concurrency : TransactionConcurrency.values()) { + EnumSet supported = EnumSet.of(TransactionIsolation.READ_COMMITTED); + + for (TransactionIsolation isolation : TransactionIsolation.values()) { + try (Transaction ignored = srv.transactions().txStart(concurrency, isolation)) { + sql("DELETE FROM T1"); + + assertTrue(supported.remove(isolation)); + } + catch (CacheException e) { + assertFalse(supported.contains(isolation)); + } + } + + assertTrue(supported.isEmpty()); + } + } + } + + /** */ + @Test + public void testJoinFailIfConfigMismatch() throws Exception { + try (IgniteEx srv = startGrid(0)) { + for (boolean client : new boolean[] {false, true}) { + assertThrows( + null, + () -> client ? startClientGrid(1) : startGrid(1), + IgniteCheckedException.class, + "Transactions aware queries enabled mismatch (fix txAwareQueriesEnabled property)"); + } + } + } + + /** */ + public List> sql(String sqlText, Object... args) { + SqlFieldsQuery qry = new SqlFieldsQuery(sqlText) + .setArgs(args) + .setTimeout(5, SECONDS); + + return grid(0).context().query().querySqlFields(qry, false, false).get(0).getAll(); + } +} diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java index e71990733db80..df909b46f17db 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IgniteCalciteTestSuite.java @@ -19,13 +19,17 @@ import org.apache.ignite.internal.processors.query.calcite.QueryCheckerTest; import org.apache.ignite.internal.processors.query.calcite.exec.ClosableIteratorsHolderTest; +import org.apache.ignite.internal.processors.query.calcite.exec.KeyFilteringCursorTest; import org.apache.ignite.internal.processors.query.calcite.exec.LogicalRelImplementorTest; import org.apache.ignite.internal.processors.query.calcite.exec.NumericTypesPrecisionsTest; import org.apache.ignite.internal.processors.query.calcite.exec.exp.IgniteSqlFunctionsTest; import org.apache.ignite.internal.processors.query.calcite.exec.tracker.MemoryTrackerTest; +import org.apache.ignite.internal.processors.query.calcite.jdbc.JdbcThinTransactionalSelfTest; import org.apache.ignite.internal.processors.query.calcite.message.CalciteCommunicationMessageSerializationTest; import org.apache.ignite.internal.processors.query.calcite.sql.SqlCustomParserTest; import org.apache.ignite.internal.processors.query.calcite.sql.SqlReservedWordsTest; +import org.apache.ignite.internal.processors.tx.SqlTransactionsIsolationTest; +import org.apache.ignite.internal.processors.tx.SqlTransactionsUnsupportedModesTest; import org.junit.runner.RunWith; import org.junit.runners.Suite; @@ -50,6 +54,12 @@ CalciteCommunicationMessageSerializationTest.class, NumericTypesPrecisionsTest.class, + + KeyFilteringCursorTest.class, + SqlTransactionsIsolationTest.class, + SqlTransactionsUnsupportedModesTest.class, + + JdbcThinTransactionalSelfTest.class, }) public class IgniteCalciteTestSuite { } diff --git a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java index 15c302e337ee1..c3f89473daf17 100644 --- a/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java +++ b/modules/calcite/src/test/java/org/apache/ignite/testsuites/IntegrationTestSuite.java @@ -67,6 +67,7 @@ import org.apache.ignite.internal.processors.query.calcite.integration.TimeoutIntegrationTest; import org.apache.ignite.internal.processors.query.calcite.integration.UserDdlIntegrationTest; import org.apache.ignite.internal.processors.query.calcite.integration.UserDefinedFunctionsIntegrationTest; +import org.apache.ignite.internal.processors.query.calcite.integration.UserDefinedFunctionsIntegrationTransactionalTest; import org.apache.ignite.internal.processors.query.calcite.integration.ViewsIntegrationTest; import org.apache.ignite.internal.processors.query.calcite.jdbc.JdbcCrossEngineTest; import org.apache.ignite.internal.processors.query.calcite.jdbc.JdbcQueryTest; @@ -118,6 +119,7 @@ JoinIntegrationTest.class, IntervalTest.class, UserDefinedFunctionsIntegrationTest.class, + UserDefinedFunctionsIntegrationTransactionalTest.class, CorrelatesIntegrationTest.class, SystemViewsIntegrationTest.class, IndexRebuildIntegrationTest.class, diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java index 06b06159282dd..14de853808fa2 100644 --- a/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java +++ b/modules/core/src/main/java/org/apache/ignite/configuration/TransactionConfiguration.java @@ -19,6 +19,8 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.EnumSet; +import java.util.Set; import javax.cache.configuration.Factory; import org.apache.ignite.internal.util.TransientSerializable; import org.apache.ignite.internal.util.typedef.internal.S; @@ -32,6 +34,13 @@ */ @TransientSerializable(methodName = "transientSerializableFields") public class TransactionConfiguration implements Serializable { + /** + * Supported levels of transaction isolation for SQL queries. + * + * @see #setTxAwareQueriesEnabled(boolean) + */ + public static final Set TX_AWARE_QUERIES_SUPPORTED_MODES = EnumSet.of(TransactionIsolation.READ_COMMITTED); + /** */ private static final IgniteProductVersion TX_PME_TIMEOUT_SINCE = IgniteProductVersion.fromString("2.5.1"); @@ -100,6 +109,12 @@ public class TransactionConfiguration implements Serializable { */ private boolean useJtaSync; + /** + * When set to true, Ignite will execute SQL and scan queries in transaction aware mode. + * Default is {@code false}. + */ + private boolean txAwareQueriesEnabled; + /** * Empty constructor. */ @@ -121,6 +136,7 @@ public TransactionConfiguration(TransactionConfiguration cfg) { tmLookupClsName = cfg.getTxManagerLookupClassName(); txManagerFactory = cfg.getTxManagerFactory(); useJtaSync = cfg.isUseJtaSynchronization(); + txAwareQueriesEnabled = cfg.isTxAwareQueriesEnabled(); } /** @@ -409,6 +425,23 @@ public TransactionConfiguration setUseJtaSynchronization(boolean useJtaSync) { return this; } + /** + * @return Whether to execute SQL and scan queries in transaction aware mode. + */ + public boolean isTxAwareQueriesEnabled() { + return txAwareQueriesEnabled; + } + + /** + * @param txAwareQueriesEnabled Whether to execute SQL and scan queries in transaction aware mode. + * @return {@code this} for chaining. + */ + public TransactionConfiguration setTxAwareQueriesEnabled(boolean txAwareQueriesEnabled) { + this.txAwareQueriesEnabled = txAwareQueriesEnabled; + + return this; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(TransactionConfiguration.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 70e3001eb9a53..a2f7251c2b3f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -265,6 +265,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SHUTDOWN_POLICY; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SPI_CLASS; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_AWARE_QUERIES_ENABLED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_SERIALIZABLE_ENABLED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME; import static org.apache.ignite.internal.IgniteVersionUtils.BUILD_TSTAMP_STR; @@ -1705,6 +1706,7 @@ private void fillNodeAttributes(boolean notifyEnabled) throws IgniteCheckedExcep // Save transactions configuration. add(ATTR_TX_SERIALIZABLE_ENABLED, cfg.getTransactionConfiguration().isTxSerializableEnabled()); + add(ATTR_TX_AWARE_QUERIES_ENABLED, cfg.getTransactionConfiguration().isTxAwareQueriesEnabled()); // Supported features. add(ATTR_IGNITE_FEATURES, IgniteFeatures.allFeatures()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java index 16b29e1a0f028..0f938884604c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java @@ -81,6 +81,9 @@ public final class IgniteNodeAttributes { /** Internal attribute name constant. */ public static final String ATTR_TX_SERIALIZABLE_ENABLED = ATTR_PREFIX + ".tx.serializable.enabled"; + /** Internal attribute name constant. */ + public static final String ATTR_TX_AWARE_QUERIES_ENABLED = ATTR_PREFIX + ".tx.aware.queries.enabled"; + /** Internal attribute name constant. */ public static final String ATTR_JMX_PORT = ATTR_PREFIX + ".jmx.port"; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java index 25ce7d8f34f36..a31a9cf8b713f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexImpl.java @@ -19,8 +19,6 @@ import java.util.ArrayDeque; import java.util.Collections; -import java.util.Comparator; -import java.util.PriorityQueue; import java.util.Queue; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; @@ -31,12 +29,9 @@ import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.cache.query.index.AbstractIndex; import org.apache.ignite.internal.cache.query.index.SingleCursor; -import org.apache.ignite.internal.cache.query.index.SortOrder; import org.apache.ignite.internal.cache.query.index.sorted.DurableBackgroundCleanupIndexTreeTaskV2; -import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; -import org.apache.ignite.internal.cache.query.index.sorted.IndexRowComparator; import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl; import org.apache.ignite.internal.cache.query.index.sorted.IndexValueCursor; import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler; @@ -170,7 +165,7 @@ public InlineIndexImpl(GridCacheContext cctx, SortedIndexDefinition def, I for (int i = 0; i < segmentsCnt; i++) segmentCursors[i] = find(lower, upper, lowIncl, upIncl, i, qryCtx); - return new SegmentedIndexCursor(segmentCursors, def); + return new SortedSegmentedIndexCursor(segmentCursors, def); } finally { lock.readLock().unlock(); @@ -593,86 +588,8 @@ private void destroy0(boolean softDel) throws IgniteCheckedException { return def; } - /** Single cursor over multiple segments. The next value is chosen with the index row comparator. */ - private static class SegmentedIndexCursor implements GridCursor { - /** Cursors over segments. */ - private final Queue> cursors; - - /** Comparator to compare index rows. */ - private final Comparator> cursorComp; - - /** */ - private IndexRow head; - - /** */ - SegmentedIndexCursor(GridCursor[] cursors, SortedIndexDefinition idxDef) throws IgniteCheckedException { - cursorComp = new Comparator>() { - private final IndexRowComparator rowComparator = idxDef.rowComparator(); - - private final IndexKeyDefinition[] keyDefs = - idxDef.indexKeyDefinitions().values().toArray(new IndexKeyDefinition[0]); - - @Override public int compare(GridCursor o1, GridCursor o2) { - try { - int keysLen = o1.get().keysCount(); - - for (int i = 0; i < keysLen; i++) { - int cmp = rowComparator.compareRow(o1.get(), o2.get(), i); - - if (cmp != 0) { - boolean desc = keyDefs[i].order().sortOrder() == SortOrder.DESC; - - return desc ? -cmp : cmp; - } - } - - return 0; - } - catch (IgniteCheckedException e) { - throw new IgniteException("Failed to sort remote index rows", e); - } - } - }; - - this.cursors = cursorsQueue(cursors); - } - - /** {@inheritDoc} */ - @Override public boolean next() throws IgniteCheckedException { - if (cursors.isEmpty()) - return false; - - GridCursor c = cursors.poll(); - - head = c.get(); - - if (c.next()) - cursors.add(c); - - return true; - } - - /** {@inheritDoc} */ - @Override public IndexRow get() throws IgniteCheckedException { - return head; - } - - /** */ - protected Queue> cursorsQueue(GridCursor[] cursors) - throws IgniteCheckedException { - PriorityQueue> q = new PriorityQueue<>(cursors.length, cursorComp); - - for (GridCursor c: cursors) { - if (c.next()) - q.add(c); - } - - return q; - } - } - /** First-only, single-value-only segmented cursor. */ - private static class SingleValueSegmentedIndexCursor extends SegmentedIndexCursor { + private static class SingleValueSegmentedIndexCursor extends SortedSegmentedIndexCursor { /** Ctor. */ SingleValueSegmentedIndexCursor( GridCursor[] cursors, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java index 2eb0b38b8675d..138137c771914 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/InlineIndexTree.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; import org.apache.ignite.internal.cache.query.index.sorted.IndexRowCache; +import org.apache.ignite.internal.cache.query.index.sorted.IndexRowComparator; import org.apache.ignite.internal.cache.query.index.sorted.IndexRowImpl; import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandler; import org.apache.ignite.internal.cache.query.index.sorted.InlineIndexRowHandlerFactory; @@ -256,7 +257,7 @@ private boolean inlineObjectSupported(SortedIndexDefinition def, MetaPageInfo me if (inlineSize == 0) { IndexRow currRow = getRow(io, pageAddr, idx); - return compareFullRows(currRow, row, 0); + return compareFullRows(currRow, row, 0, rowHandler(), def.rowComparator()); } int fieldOff = 0; @@ -306,27 +307,35 @@ private boolean inlineObjectSupported(SortedIndexDefinition def, MetaPageInfo me if (currRow == null) currRow = getRow(io, pageAddr, idx); - return compareFullRows(currRow, row, keyIdx); + return compareFullRows(currRow, row, keyIdx, rowHandler(), def.rowComparator()); } return 0; } /** */ - private int compareFullRows(IndexRow currRow, IndexRow row, int from) throws IgniteCheckedException { + public static int compareFullRows( + IndexRow currRow, + IndexRow row, + int from, + InlineIndexRowHandler rowHnd, + IndexRowComparator rowCmp + ) throws IgniteCheckedException { if (currRow == row) return 0; - for (int i = from; i < rowHandler().indexKeyDefinitions().size(); i++) { + List idxKeyDefs = rowHnd.indexKeyDefinitions(); + + for (int i = from; i < idxKeyDefs.size(); i++) { // 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.key(i) == null) return 0; - int c = def.rowComparator().compareRow(currRow, row, i); + int c = rowCmp.compareRow(currRow, row, i); if (c != 0) - return applySortOrder(Integer.signum(c), rowHnd.indexKeyDefinitions().get(i).order().sortOrder()); + return applySortOrder(Integer.signum(c), idxKeyDefs.get(i).order().sortOrder()); } return 0; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/SortedSegmentedIndexCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/SortedSegmentedIndexCursor.java new file mode 100644 index 0000000000000..5384cff40a35f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/SortedSegmentedIndexCursor.java @@ -0,0 +1,108 @@ +/* + * 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.Comparator; +import java.util.PriorityQueue; +import java.util.Queue; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.cache.query.index.SortOrder; +import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; +import org.apache.ignite.internal.cache.query.index.sorted.IndexRow; +import org.apache.ignite.internal.cache.query.index.sorted.IndexRowComparator; +import org.apache.ignite.internal.cache.query.index.sorted.SortedIndexDefinition; +import org.apache.ignite.internal.util.lang.GridCursor; + +/** Single cursor over multiple segments. The next value is chosen with the index row comparator. */ +public class SortedSegmentedIndexCursor implements GridCursor { + /** Cursors over segments. */ + private final Queue> cursors; + + /** Comparator to compare index rows. */ + private final Comparator> cursorComp; + + /** */ + private IndexRow head; + + /** */ + public SortedSegmentedIndexCursor(GridCursor[] cursors, SortedIndexDefinition idxDef) throws IgniteCheckedException { + cursorComp = new Comparator<>() { + private final IndexRowComparator rowComparator = idxDef.rowComparator(); + + private final IndexKeyDefinition[] keyDefs = + idxDef.indexKeyDefinitions().values().toArray(new IndexKeyDefinition[0]); + + @Override public int compare(GridCursor o1, GridCursor o2) { + try { + int keysLen = o1.get().keysCount(); + + for (int i = 0; i < keysLen; i++) { + int cmp = rowComparator.compareRow(o1.get(), o2.get(), i); + + if (cmp != 0) { + boolean desc = keyDefs[i].order().sortOrder() == SortOrder.DESC; + + return desc ? -cmp : cmp; + } + } + + return 0; + } + catch (IgniteCheckedException e) { + throw new IgniteException("Failed to sort remote index rows", e); + } + } + }; + + this.cursors = cursorsQueue(cursors); + } + + /** {@inheritDoc} */ + @Override public boolean next() throws IgniteCheckedException { + if (cursors.isEmpty()) + return false; + + GridCursor c = cursors.poll(); + + head = c.get(); + + if (c.next()) + cursors.add(c); + + return true; + } + + /** {@inheritDoc} */ + @Override public IndexRow get() throws IgniteCheckedException { + return head; + } + + /** */ + protected Queue> cursorsQueue(GridCursor[] cursors) + throws IgniteCheckedException { + PriorityQueue> q = new PriorityQueue<>(cursors.length, cursorComp); + + for (GridCursor c: cursors) { + if (c.next()) + q.add(c); + } + + return q; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientFieldsQueryPager.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientFieldsQueryPager.java index f7c080f78371f..157de864383e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientFieldsQueryPager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientFieldsQueryPager.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.function.Consumer; import org.apache.ignite.internal.binary.streams.BinaryInputStream; +import org.apache.ignite.internal.client.thin.TcpClientTransactions.TcpClientTransaction; +import org.jetbrains.annotations.Nullable; /** * Fields query pager. @@ -39,13 +41,14 @@ class ClientFieldsQueryPager extends GenericQueryPager> implements Field /** Constructor. */ ClientFieldsQueryPager( ReliableChannel ch, + @Nullable TcpClientTransaction tx, ClientOperation qryOp, ClientOperation pageQryOp, Consumer qryWriter, boolean keepBinary, ClientBinaryMarshaller marsh ) { - super(ch, qryOp, pageQryOp, qryWriter); + super(ch, tx, qryOp, pageQryOp, qryWriter); this.keepBinary = keepBinary; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryPager.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryPager.java index 622a2c8cafa06..ffbe7184288e2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryPager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryPager.java @@ -21,6 +21,8 @@ import java.util.function.Consumer; import javax.cache.Cache; import org.apache.ignite.internal.binary.streams.BinaryInputStream; +import org.apache.ignite.internal.client.thin.TcpClientTransactions.TcpClientTransaction; +import org.jetbrains.annotations.Nullable; /** * Client query pager. @@ -35,13 +37,14 @@ class ClientQueryPager extends GenericQueryPager> { /** Constructor. */ ClientQueryPager( ReliableChannel ch, + @Nullable TcpClientTransaction tx, ClientOperation qryOp, ClientOperation pageQryOp, Consumer qryWriter, boolean keepBinary, ClientBinaryMarshaller marsh ) { - super(ch, qryOp, pageQryOp, qryWriter); + super(ch, tx, qryOp, pageQryOp, qryWriter); this.keepBinary = keepBinary; @@ -51,6 +54,7 @@ class ClientQueryPager extends GenericQueryPager> { /** Constructor. */ ClientQueryPager( ReliableChannel ch, + @Nullable TcpClientTransaction tx, ClientOperation qryOp, ClientOperation pageQryOp, Consumer qryWriter, @@ -59,7 +63,7 @@ class ClientQueryPager extends GenericQueryPager> { int cacheId, int part ) { - super(ch, qryOp, pageQryOp, qryWriter, cacheId, part); + super(ch, tx, qryOp, pageQryOp, qryWriter, cacheId, part); this.keepBinary = keepBinary; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/GenericQueryPager.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/GenericQueryPager.java index 9bc35417db905..a53e48b1b0315 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/GenericQueryPager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/GenericQueryPager.java @@ -19,10 +19,11 @@ import java.util.Collection; import java.util.function.Consumer; - import org.apache.ignite.client.ClientConnectionException; import org.apache.ignite.client.ClientException; import org.apache.ignite.client.ClientReconnectedException; +import org.apache.ignite.internal.client.thin.TcpClientTransactions.TcpClientTransaction; +import org.jetbrains.annotations.Nullable; /** * Generic query pager. Override {@link this#readResult(PayloadInputChannel)} to make it specific. @@ -40,6 +41,9 @@ abstract class GenericQueryPager implements QueryPager { /** Channel. */ private final ReliableChannel ch; + /** Client Transaction. */ + private final @Nullable TcpClientTransaction tx; + /** Has next. */ private boolean hasNext = true; @@ -61,6 +65,7 @@ abstract class GenericQueryPager implements QueryPager { /** Constructor. */ GenericQueryPager( ReliableChannel ch, + @Nullable TcpClientTransaction tx, ClientOperation qryOp, ClientOperation pageQryOp, Consumer qryWriter, @@ -68,6 +73,7 @@ abstract class GenericQueryPager implements QueryPager { int part ) { this.ch = ch; + this.tx = tx; this.qryOp = qryOp; this.pageQryOp = pageQryOp; this.qryWriter = qryWriter; @@ -78,11 +84,12 @@ abstract class GenericQueryPager implements QueryPager { /** Constructor. */ GenericQueryPager( ReliableChannel ch, + @Nullable TcpClientTransaction tx, ClientOperation qryOp, ClientOperation pageQryOp, Consumer qryWriter ) { - this(ch, qryOp, pageQryOp, qryWriter, 0, -1); + this(ch, tx, qryOp, pageQryOp, qryWriter, 0, -1); } /** {@inheritDoc} */ @@ -90,8 +97,15 @@ abstract class GenericQueryPager implements QueryPager { if (!hasNext) throw new IllegalStateException("No more query results"); - return hasFirstPage ? queryPage() : part == -1 ? ch.service(qryOp, qryWriter, this::readResult) : - ch.affinityService(cacheId, part, qryOp, qryWriter, this::readResult); + if (hasFirstPage) + return queryPage(); + + if (tx != null && tx.clientChannel().protocolCtx().isFeatureSupported(ProtocolBitmaskFeature.TX_AWARE_QUERIES)) + return tx.clientChannel().service(qryOp, qryWriter, this::readResult); + + return part == -1 + ? ch.service(qryOp, qryWriter, this::readResult) + : ch.affinityService(cacheId, part, qryOp, qryWriter, this::readResult); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java index 25c22ce89698d..1a0fb9d714829 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java @@ -80,7 +80,10 @@ public enum ProtocolBitmaskFeature { SERVICE_TOPOLOGY(16), /** Cache invoke/invokeAll operations. */ - CACHE_INVOKE(17); + CACHE_INVOKE(17), + + /** Transaction aware queries. */ + TX_AWARE_QUERIES(18); /** */ private static final EnumSet ALL_FEATURES_AS_ENUM_SET = diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java index 3d280b40d7ad0..f1966f4c1f7d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientCache.java @@ -80,10 +80,10 @@ */ public class TcpClientCache implements ClientCache { /** "Keep binary" flag mask. */ - private static final byte KEEP_BINARY_FLAG_MASK = 0x01; + static final byte KEEP_BINARY_FLAG_MASK = 0x01; /** "Transactional" flag mask. */ - private static final byte TRANSACTIONAL_FLAG_MASK = 0x02; + static final byte TRANSACTIONAL_FLAG_MASK = 0x02; /** "With expiry policy" flag mask. */ private static final byte WITH_EXPIRY_POLICY_FLAG_MASK = 0x04; @@ -973,12 +973,18 @@ else if (qry instanceof IndexQuery) throw new NullPointerException("qry"); Consumer qryWriter = payloadCh -> { - writeCacheInfo(payloadCh); + writeCacheInfo( + payloadCh, + payloadCh.clientChannel().protocolCtx().isFeatureSupported(ProtocolBitmaskFeature.TX_AWARE_QUERIES) + ? transactions.tx() + : null + ); serDes.write(qry, payloadCh.out()); }; return new ClientFieldsQueryCursor<>(new ClientFieldsQueryPager( ch, + transactions.tx(), ClientOperation.QUERY_SQL_FIELDS, ClientOperation.QUERY_SQL_FIELDS_CURSOR_GET_PAGE, qryWriter, @@ -1149,6 +1155,7 @@ private QueryCursor> scanQuery(ScanQuery qry) { return new ClientQueryCursor<>(new ClientQueryPager<>( ch, + null, ClientOperation.QUERY_SCAN, ClientOperation.QUERY_SCAN_CURSOR_GET_PAGE, qryWriter, @@ -1235,6 +1242,7 @@ else if (c instanceof InIndexQueryCriterion) { return new ClientQueryCursor<>(new ClientQueryPager<>( ch, + null, ClientOperation.QUERY_INDEX, ClientOperation.QUERY_INDEX_CURSOR_GET_PAGE, qryWriter, @@ -1264,6 +1272,7 @@ private QueryCursor> sqlQuery(SqlQuery qry) { return new ClientQueryCursor<>(new ClientQueryPager<>( ch, + null, ClientOperation.QUERY_SQL, ClientOperation.QUERY_SQL_CURSOR_GET_PAGE, qryWriter, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java index 48c490198cb2f..ae2ac0c42674a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java @@ -69,6 +69,7 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl; import org.apache.ignite.internal.binary.streams.BinaryInputStream; import org.apache.ignite.internal.binary.streams.BinaryOutputStream; +import org.apache.ignite.internal.client.thin.TcpClientTransactions.TcpClientTransaction; import org.apache.ignite.internal.client.thin.io.ClientConnectionMultiplexer; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; @@ -323,13 +324,33 @@ private TcpIgniteClient(ClientConfiguration cfg) throws ClientException { Consumer qryWriter = payloadCh -> { BinaryOutputStream out = payloadCh.out(); + byte flags = TcpClientCache.KEEP_BINARY_FLAG_MASK; + + int txId = 0; + + if (payloadCh.clientChannel().protocolCtx().isFeatureSupported(ProtocolBitmaskFeature.TX_AWARE_QUERIES)) { + TcpClientTransaction tx = transactions.tx(); + + txId = tx == null ? 0 : tx.txId(); + } + out.writeInt(0); // no cache ID - out.writeByte((byte)1); // keep binary + + if (txId != 0) { + flags |= TcpClientCache.TRANSACTIONAL_FLAG_MASK; + + out.writeByte(flags); + out.writeInt(txId); + } + else + out.writeByte(flags); + serDes.write(qry, out); }; return new ClientFieldsQueryCursor<>(new ClientFieldsQueryPager( ch, + transactions.tx(), ClientOperation.QUERY_SQL_FIELDS, ClientOperation.QUERY_SQL_FIELDS_CURSOR_GET_PAGE, qryWriter, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java index c49f28682168f..30b8809ab80b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionProperties.java @@ -20,6 +20,7 @@ import java.sql.SQLException; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcThinFeature; import org.apache.ignite.internal.util.HostAndPortRange; +import org.apache.ignite.transactions.TransactionConcurrency; import org.jetbrains.annotations.Nullable; /** @@ -560,4 +561,40 @@ public void setPartitionAwarenessPartitionDistributionsCacheSize( * @param qryEngine SQL Query engine name. */ public void setQueryEngine(String qryEngine); + + /** + * @return Transaction concurrency value. + */ + public TransactionConcurrency getTransactionConcurrency(); + + /** + * Sets transaction concurrency. + * + * @param transactionConcurrency Transaction concurrecny. + */ + public void setTransactionConcurrency(String transactionConcurrency); + + /** + * @return Transaction timeout in milliseconds. + */ + public int getTransactionTimeout(); + + /** + * Sets transaction timeout in milliseconds. + * + * @param transactionTimeout Transaction timeout in millicesonds. + */ + public void setTransactionTimeout(int transactionTimeout) throws SQLException; + + /** + * @return Transaction label. + */ + public String getTransactionLabel(); + + /** + * Sets transaction label. + * + * @param transactionLabel Transaction label. + */ + public void setTransactionLabel(String transactionLabel); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java index f69d907973f50..0b933d4261d11 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java @@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcThinFeature; import org.apache.ignite.internal.util.HostAndPortRange; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.transactions.TransactionConcurrency; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.cache.query.SqlFieldsQuery.DFLT_LAZY; @@ -56,167 +57,167 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa private HostAndPortRange[] addrs; /** Schema name. Hidden property. Is used to set default schema name part of the URL. */ - private StringProperty schema = new StringProperty(PROP_SCHEMA, + private final StringProperty schema = new StringProperty(PROP_SCHEMA, "Schema name of the connection", "PUBLIC", null, false, null); /** Distributed joins property. */ - private BooleanProperty distributedJoins = new BooleanProperty( + private final BooleanProperty distributedJoins = new BooleanProperty( "distributedJoins", "Enable distributed joins", false, false); /** Enforce join order property. */ - private BooleanProperty enforceJoinOrder = new BooleanProperty( + private final BooleanProperty enforceJoinOrder = new BooleanProperty( "enforceJoinOrder", "Enable enforce join order", false, false); /** Collocated property. */ - private BooleanProperty collocated = new BooleanProperty( + private final BooleanProperty collocated = new BooleanProperty( "collocated", "Enable collocated query", false, false); /** Replicated only property. */ - private BooleanProperty replicatedOnly = new BooleanProperty( + private final BooleanProperty replicatedOnly = new BooleanProperty( "replicatedOnly", "Specify if the all queries contain only replicated tables", false, false); /** Auto close server cursor property. */ - private BooleanProperty autoCloseServerCursor = new BooleanProperty( + private final BooleanProperty autoCloseServerCursor = new BooleanProperty( "autoCloseServerCursor", "Enable auto close server cursors when last piece of result set is retrieved. " + "If the server-side cursor is already closed, you may get an exception when trying to call " + "`ResultSet.getMetadata()` method.", false, false); /** TCP no delay property. */ - private BooleanProperty tcpNoDelay = new BooleanProperty( + private final BooleanProperty tcpNoDelay = new BooleanProperty( "tcpNoDelay", "TCP no delay flag", true, false); /** Lazy query execution property. */ - private BooleanProperty lazy = new BooleanProperty( + private final BooleanProperty lazy = new BooleanProperty( "lazy", "Enable lazy query execution", DFLT_LAZY, false); /** Socket send buffer size property. */ - private IntegerProperty socketSendBuffer = new IntegerProperty( + private final IntegerProperty socketSendBuffer = new IntegerProperty( "socketSendBuffer", "Socket send buffer size", DFLT_SOCK_BUFFER_SIZE, false, 0, Integer.MAX_VALUE); /** Socket receive buffer size property. */ - private IntegerProperty socketReceiveBuffer = new IntegerProperty( + private final IntegerProperty socketReceiveBuffer = new IntegerProperty( "socketReceiveBuffer", "Socket send buffer size", DFLT_SOCK_BUFFER_SIZE, false, 0, Integer.MAX_VALUE); /** Executes update queries on ignite server nodes flag. */ - private BooleanProperty skipReducerOnUpdate = new BooleanProperty( + private final BooleanProperty skipReducerOnUpdate = new BooleanProperty( "skipReducerOnUpdate", "Enable execution update queries on ignite server nodes", false, false); /** SSL: Use SSL connection to Ignite node. */ - private StringProperty sslMode = new StringProperty("sslMode", + private final StringProperty sslMode = new StringProperty("sslMode", "The SSL mode of the connection", SSL_MODE_DISABLE, new String[] {SSL_MODE_DISABLE, SSL_MODE_REQUIRE}, false, null); /** SSL: Client certificate key store url. */ - private StringProperty sslProtocol = new StringProperty("sslProtocol", + private final StringProperty sslProtocol = new StringProperty("sslProtocol", "SSL protocol name", null, null, false, null); /** SSL: Supported SSL cipher suites. */ - private StringProperty sslCipherSuites = new StringProperty("sslCipherSuites", + private final StringProperty sslCipherSuites = new StringProperty("sslCipherSuites", "Supported SSL ciphers", null, null, false, null); /** SSL: Key algorithm name. */ - private StringProperty sslKeyAlgorithm = new StringProperty("sslKeyAlgorithm", + private final StringProperty sslKeyAlgorithm = new StringProperty("sslKeyAlgorithm", "SSL key algorithm name", null, null, false, null); /** SSL: Client certificate key store url. */ - private StringProperty sslClientCertificateKeyStoreUrl = + private final StringProperty sslClientCertificateKeyStoreUrl = new StringProperty("sslClientCertificateKeyStoreUrl", "Client certificate key store URL", null, null, false, null); /** SSL: Client certificate key store password. */ - private StringProperty sslClientCertificateKeyStorePassword = + private final StringProperty sslClientCertificateKeyStorePassword = new StringProperty("sslClientCertificateKeyStorePassword", "Client certificate key store password", null, null, false, null); /** SSL: Client certificate key store type. */ - private StringProperty sslClientCertificateKeyStoreType = + private final StringProperty sslClientCertificateKeyStoreType = new StringProperty("sslClientCertificateKeyStoreType", "Client certificate key store type", null, null, false, null); /** SSL: Trusted certificate key store url. */ - private StringProperty sslTrustCertificateKeyStoreUrl = + private final StringProperty sslTrustCertificateKeyStoreUrl = new StringProperty("sslTrustCertificateKeyStoreUrl", "Trusted certificate key store URL", null, null, false, null); /** SSL Trusted certificate key store password. */ - private StringProperty sslTrustCertificateKeyStorePassword = + private final StringProperty sslTrustCertificateKeyStorePassword = new StringProperty("sslTrustCertificateKeyStorePassword", "Trusted certificate key store password", null, null, false, null); /** SSL: Trusted certificate key store type. */ - private StringProperty sslTrustCertificateKeyStoreType = + private final StringProperty sslTrustCertificateKeyStoreType = new StringProperty("sslTrustCertificateKeyStoreType", "Trusted certificate key store type", null, null, false, null); /** SSL: Trust all certificates. */ - private BooleanProperty sslTrustAll = new BooleanProperty("sslTrustAll", + private final BooleanProperty sslTrustAll = new BooleanProperty("sslTrustAll", "Trust all certificates", false, false); /** SSL: Custom class name that implements Factory<SSLSocketFactory>. */ - private StringProperty sslFactory = new StringProperty("sslFactory", + private final StringProperty sslFactory = new StringProperty("sslFactory", "Custom class name that implements Factory", null, null, false, null); /** Custom class name that implements Factory<Map<String, String>> which returns user attributes. */ - private StringProperty userAttrsFactory = new StringProperty("userAttributesFactory", + private final StringProperty userAttrsFactory = new StringProperty("userAttributesFactory", "Custom class name that implements Factory> (user attributes)", null, null, false, null); /** User name to authenticate the client on the server side. */ - private StringProperty user = new StringProperty( + private final StringProperty user = new StringProperty( "user", "User name to authenticate the client on the server side", null, null, false, null); /** User's password. */ - private StringProperty passwd = new StringProperty( + private final StringProperty passwd = new StringProperty( "password", "User's password", null, null, false, null); /** Data page scan flag. */ - private BooleanProperty dataPageScanEnabled = new BooleanProperty("dataPageScanEnabled", + private final BooleanProperty dataPageScanEnabled = new BooleanProperty("dataPageScanEnabled", "Whether data page scan for queries is allowed. If not specified, server defines the default behaviour.", null, false); /** Partition awareness flag. */ - private BooleanProperty partitionAwareness = new BooleanProperty( + private final BooleanProperty partitionAwareness = new BooleanProperty( "partitionAwareness", "Whether jdbc thin partition awareness is enabled.", false, false); /** Update batch size (the size of internal batches are used for INSERT/UPDATE/DELETE operation). */ - private IntegerProperty updateBatchSize = new IntegerProperty("updateBatchSize", + private final IntegerProperty updateBatchSize = new IntegerProperty("updateBatchSize", "Update bach size (the size of internal batches are used for INSERT/UPDATE/DELETE operation). " + "Set to 1 to prevent deadlock on update where keys sequence are different " + "in several concurrent updates.", null, false, 1, Integer.MAX_VALUE); /** Partition awareness SQL cache size. */ - private IntegerProperty partitionAwarenessSQLCacheSize = new IntegerProperty("partitionAwarenessSQLCacheSize", + private final IntegerProperty partitionAwarenessSQLCacheSize = new IntegerProperty("partitionAwarenessSQLCacheSize", "The size of sql cache that is used within partition awareness optimization.", 1_000, false, 1, Integer.MAX_VALUE); /** Partition awareness partition distributions cache size. */ - private IntegerProperty partitionAwarenessPartDistributionsCacheSize = new IntegerProperty( + private final IntegerProperty partitionAwarenessPartDistributionsCacheSize = new IntegerProperty( "partitionAwarenessPartitionDistributionsCacheSize", "The size of partition distributions cache that is used within partition awareness optimization.", 1_000, false, 1, Integer.MAX_VALUE); /** Query timeout. */ - private IntegerProperty qryTimeout = new IntegerProperty("queryTimeout", + private final IntegerProperty qryTimeout = new IntegerProperty("queryTimeout", "Sets the number of seconds the driver will wait for a Statement object to execute." + " Zero means there is no limits.", null, false, 0, Integer.MAX_VALUE); /** JDBC connection timeout. */ - private IntegerProperty connTimeout = new IntegerProperty("connectionTimeout", + private final IntegerProperty connTimeout = new IntegerProperty("connectionTimeout", "Sets the number of milliseconds JDBC client will waits for server to response." + " Zero means there is no limits.", 0, false, 0, Integer.MAX_VALUE); /** Disabled features. */ - private StringProperty disabledFeatures = new StringProperty("disabledFeatures", + private final StringProperty disabledFeatures = new StringProperty("disabledFeatures", "Sets enumeration of features to force disable its.", null, null, @@ -240,13 +241,30 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa }); /** Keep binary objects in binary form. */ - private BooleanProperty keepBinary = new BooleanProperty("keepBinary", + private final BooleanProperty keepBinary = new BooleanProperty("keepBinary", "Whether to keep binary objects in binary form.", false, false); /** Use specified SQL query engine for a connection. */ private final StringProperty qryEngine = new StringProperty("queryEngine", "Use specified SQL query engine for a connection.", null, null, false, null); + /** Transaction concurrency. */ + private final StringProperty transactionConcurrency = new StringProperty("transactionConcurrency", + "Transaction concurrencty level.", + TransactionConcurrency.OPTIMISTIC.name(), + new String[]{TransactionConcurrency.OPTIMISTIC.name(), TransactionConcurrency.PESSIMISTIC.name()}, + false, + null + ); + + /** JDBC transaction timeout. */ + private final IntegerProperty transactionTimeout = new IntegerProperty("transactionTimeout", + "Sets the number of milliseconds for server-side transaction timeout. Zero means there is no limits.", + 0, false, 0, Integer.MAX_VALUE); + + /** Transaction label. */ + private final StringProperty transactionLabel = new StringProperty("transactionLabel", "Transaction label.", null, null, false, null); + /** Properties array. */ private final ConnectionProperty[] propsArr = { distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseServerCursor, @@ -266,7 +284,10 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa connTimeout, disabledFeatures, keepBinary, - qryEngine + qryEngine, + transactionConcurrency, + transactionTimeout, + transactionLabel }; /** {@inheritDoc} */ @@ -675,6 +696,36 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa this.qryEngine.setValue(qryEngine); } + /** {@inheritDoc} */ + @Override public TransactionConcurrency getTransactionConcurrency() { + return TransactionConcurrency.valueOf(transactionConcurrency.value()); + } + + /** {@inheritDoc} */ + @Override public void setTransactionConcurrency(String transactionConcurrency) { + this.transactionConcurrency.setValue(transactionConcurrency); + } + + /** {@inheritDoc} */ + @Override public int getTransactionTimeout() { + return transactionTimeout.value(); + } + + /** {@inheritDoc} */ + @Override public void setTransactionTimeout(int transactionTimeout) throws SQLException { + this.transactionTimeout.setValue(transactionTimeout); + } + + /** {@inheritDoc} */ + @Override public String getTransactionLabel() { + return transactionLabel.value(); + } + + /** {@inheritDoc} */ + @Override public void setTransactionLabel(String transactionLabel) { + this.transactionLabel.setValue(transactionLabel); + } + /** * @param url URL connection. * @param props Environment properties. @@ -1162,7 +1213,7 @@ private abstract static class NumberProperty extends ConnectionProperty { protected Number val; /** Allowed value range. */ - private Number[] range; + private final Number[] range; /** * @param name Name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/HandshakeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/HandshakeResult.java index a19cf1c60cab1..195e970355e5c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/HandshakeResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/HandshakeResult.java @@ -17,11 +17,14 @@ package org.apache.ignite.internal.jdbc.thin; +import java.util.Collections; import java.util.EnumSet; +import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcThinFeature; import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.transactions.TransactionIsolation; /** * Handshake result. @@ -39,6 +42,9 @@ class HandshakeResult { /** Features. */ private EnumSet features = EnumSet.noneOf(JdbcThinFeature.class); + /** Transaction isolation levels supported by the server for SQL queries. */ + private Set isolationLevelsSupported = Collections.emptySet(); + /** * @return Ignite server version. */ @@ -94,4 +100,18 @@ public void features(EnumSet features) { public EnumSet features() { return features; } + + /** + * @param isolationLevelsSupported Transaction isolation levels supported by the server. + */ + public void isolationLevelsSupported(Set isolationLevelsSupported) { + this.isolationLevelsSupported = isolationLevelsSupported; + } + + /** + * @return Transaction isolation levels supported by the server. + */ + public Set isolationLevelsSupported() { + return isolationLevelsSupported; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java index c5a905ac0649b..5f68115abbac8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinConnection.java @@ -48,6 +48,7 @@ import java.util.IdentityHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Random; import java.util.Set; @@ -109,7 +110,9 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResponse; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResult; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcResultWithIo; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcSetTxParametersRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType; +import org.apache.ignite.internal.processors.odbc.jdbc.JdbcTxEndRequest; import org.apache.ignite.internal.processors.odbc.jdbc.JdbcUpdateBinarySchemaResult; import org.apache.ignite.internal.sql.command.SqlCommand; import org.apache.ignite.internal.sql.command.SqlSetStreamingCommand; @@ -125,6 +128,7 @@ import org.apache.ignite.marshaller.MarshallerContext; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.thread.IgniteThreadFactory; +import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; import static java.sql.ResultSet.CLOSE_CURSORS_AT_COMMIT; @@ -147,6 +151,9 @@ public class JdbcThinConnection implements Connection { /** Logger. */ private static final Logger LOG = Logger.getLogger(JdbcThinConnection.class.getName()); + /** */ + private static final String HOLDABILITY_ERR_MSG = "Invalid holdability (can't hold cursor over commit)."; + /** Request timeout period. */ private static final int REQUEST_TIMEOUT_PERIOD = 1_000; @@ -162,6 +169,9 @@ public class JdbcThinConnection implements Connection { /** Zero timeout as query timeout means no timeout. */ static final int NO_TIMEOUT = 0; + /** No transaction id. */ + public static final int NONE_TX = 0; + /** Index generator. */ private static final AtomicLong IDX_GEN = new AtomicLong(); @@ -171,6 +181,9 @@ public class JdbcThinConnection implements Connection { /** No retries. */ public static final int NO_RETRIES = 0; + /** Default isolation level. */ + public static final int DFLT_ISOLATION = TRANSACTION_READ_COMMITTED; + /** Partition awareness enabled flag. */ private final boolean partitionAwareness; @@ -228,8 +241,11 @@ public class JdbcThinConnection implements Connection { /** Mutex. */ private final Object mux = new Object(); - /** Ignite endpoint to use within transactional context. */ - private volatile JdbcThinTcpIo txIo; + /** Transactional context. */ + private TxContext txCtx; + + /** */ + boolean isTxAwareQueriesSupported; /** Random generator. */ private static final Random RND = new Random(System.currentTimeMillis()); @@ -273,9 +289,7 @@ public JdbcThinConnection(ConnectionProperties connProps) throws SQLException { metaHnd = new JdbcBinaryMetadataHandler(); marshCtx = new JdbcMarshallerContext(); ctx = createBinaryCtx(metaHnd, marshCtx); - holdability = HOLD_CURSORS_OVER_COMMIT; autoCommit = true; - txIsolation = Connection.TRANSACTION_NONE; netTimeout = connProps.getConnectionTimeout(); qryTimeout = connProps.getQueryTimeout(); maintenanceExecutor = Executors.newScheduledThreadPool(2, @@ -296,6 +310,9 @@ public JdbcThinConnection(ConnectionProperties connProps) throws SQLException { baseEndpointVer = null; } + + holdability = isTxAwareQueriesSupported ? CLOSE_CURSORS_AT_COMMIT : HOLD_CURSORS_OVER_COMMIT; + txIsolation = defaultTransactionIsolation(); } /** Create new binary context. */ @@ -338,6 +355,67 @@ boolean isStream() { return streamState != null; } + /** @return {@code True} if transactions supported by the server, {@code false} otherwise. */ + boolean txSupportedByServer() { + return isTxAwareQueriesSupported; + } + + /** @return {@code True} if certain isolation level supported by the server, {@code false} otherwise. */ + boolean isolationLevelSupported(int level) throws SQLException { + if (level == TRANSACTION_NONE) + return true; + + if (level == TRANSACTION_READ_UNCOMMITTED) + return false; + + return defaultIo().isIsolationLevelSupported(isolation(level)); + } + + /** @return Default isolation level. */ + int defaultTransactionIsolation() { + return isTxAwareQueriesSupported ? DFLT_ISOLATION : TRANSACTION_NONE; + } + + /** @return Default io to make a request. */ + private JdbcThinTcpIo defaultIo() { + return partitionAwareness ? ios.firstEntry().getValue() : singleIo; + } + + /** @return {@code True} if transaction support currently enabled for connection. */ + boolean txEnabledForConnection() { + return isTxAwareQueriesSupported && txIsolation != TRANSACTION_NONE; + } + + /** */ + void endTransactionIfExists(boolean commit) throws SQLException { + if (txCtx == null) + return; + + txCtx.end(commit); + + txCtx = null; + } + + /** Updates transaction parameters on all known servers. */ + private void updateTransactionParameters() throws SQLException { + if (!isTxAwareQueriesSupported) + return; + + JdbcSetTxParametersRequest req = new JdbcSetTxParametersRequest( + connProps.getTransactionConcurrency(), + txIsolation == TRANSACTION_NONE ? null : isolation(txIsolation), + connProps.getTransactionTimeout(), + connProps.getTransactionLabel() + ); + + if (partitionAwareness) { + for (JdbcThinTcpIo io : ios.values()) + sendRequest(req, null, io); + } + else + sendRequest(req, null, singleIo); + } + /** * @param sql Statement. * @param cmd Parsed form of {@code sql}. @@ -346,6 +424,9 @@ boolean isStream() { */ void executeNative(String sql, SqlCommand cmd, JdbcThinStatement stmt) throws SQLException { if (cmd instanceof SqlSetStreamingCommand) { + if (txCtx != null) + throw new SQLException("Can't change stream mode inside transaction"); + SqlSetStreamingCommand cmd0 = (SqlSetStreamingCommand)cmd; // If streaming is already on, we have to close it first. @@ -371,7 +452,7 @@ void executeNative(String sql, SqlCommand cmd, JdbcThinStatement stmt) throws SQ streamState = new StreamState((SqlSetStreamingCommand)cmd, cliIo); sendRequest(new JdbcQueryExecuteRequest(JdbcStatementType.ANY_STATEMENT_TYPE, - schema, 1, 1, autoCommit, stmt.explicitTimeout, sql, null), stmt, cliIo); + schema, 1, 1, autoCommit, stmt.explicitTimeout, sql, null, NONE_TX), stmt, cliIo); streamState.start(); } @@ -409,7 +490,7 @@ void addBatch(String sql, List args) throws SQLException { int resSetHoldability) throws SQLException { ensureNotClosed(); - checkCursorOptions(resSetType, resSetConcurrency); + checkCursorOptions(resSetType, resSetConcurrency, resSetHoldability); JdbcThinStatement stmt = new JdbcThinStatement(this, resSetHoldability, schema); @@ -425,13 +506,13 @@ void addBatch(String sql, List args) throws SQLException { /** {@inheritDoc} */ @Override public PreparedStatement prepareStatement(String sql) throws SQLException { - return prepareStatement(sql, TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, HOLD_CURSORS_OVER_COMMIT); + return prepareStatement(sql, TYPE_FORWARD_ONLY, CONCUR_READ_ONLY, holdability); } /** {@inheritDoc} */ @Override public PreparedStatement prepareStatement(String sql, int resSetType, int resSetConcurrency) throws SQLException { - return prepareStatement(sql, resSetType, resSetConcurrency, HOLD_CURSORS_OVER_COMMIT); + return prepareStatement(sql, resSetType, resSetConcurrency, holdability); } /** {@inheritDoc} */ @@ -439,7 +520,7 @@ void addBatch(String sql, List args) throws SQLException { int resSetHoldability) throws SQLException { ensureNotClosed(); - checkCursorOptions(resSetType, resSetConcurrency); + checkCursorOptions(resSetType, resSetConcurrency, resSetHoldability); if (sql == null) throw new SQLException("SQL string cannot be null."); @@ -456,14 +537,18 @@ void addBatch(String sql, List args) throws SQLException { /** * @param resSetType Cursor option. * @param resSetConcurrency Cursor option. + * @param resSetHoldability Cursor option. * @throws SQLException If options unsupported. */ - private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQLException { + private void checkCursorOptions(int resSetType, int resSetConcurrency, int resSetHoldability) throws SQLException { if (resSetType != TYPE_FORWARD_ONLY) throw new SQLFeatureNotSupportedException("Invalid result set type (only forward is supported)."); if (resSetConcurrency != CONCUR_READ_ONLY) throw new SQLFeatureNotSupportedException("Invalid concurrency (updates are not supported)."); + + if (txEnabledForConnection() && resSetHoldability == HOLD_CURSORS_OVER_COMMIT) + throw new SQLFeatureNotSupportedException(HOLDABILITY_ERR_MSG); } /** {@inheritDoc} */ @@ -495,10 +580,13 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ @Override public void setAutoCommit(boolean autoCommit) throws SQLException { ensureNotClosed(); + if (txCtx != null && this.autoCommit != autoCommit) + throw new SQLException("Can't change autoCommit mode when transaction open"); + this.autoCommit = autoCommit; if (!autoCommit) - LOG.warning("Transactions are not supported."); + logTransactionWarning(); } /** {@inheritDoc} */ @@ -506,7 +594,7 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ ensureNotClosed(); if (!autoCommit) - LOG.warning("Transactions are not supported."); + logTransactionWarning(); return autoCommit; } @@ -518,7 +606,10 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ if (autoCommit) throw new SQLException("Transaction cannot be committed explicitly in auto-commit mode."); - LOG.warning("Transactions are not supported."); + if (txEnabledForConnection()) + endTransactionIfExists(true); + else + logTransactionWarning(); } /** {@inheritDoc} */ @@ -528,7 +619,10 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ if (autoCommit) throw new SQLException("Transaction cannot be rolled back explicitly in auto-commit mode."); - LOG.warning("Transactions are not supported."); + if (txEnabledForConnection()) + endTransactionIfExists(false); + else + logTransactionWarning(); } /** {@inheritDoc} */ @@ -538,6 +632,9 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ closed = true; + if (txEnabledForConnection()) + endTransactionIfExists(false); + maintenanceExecutor.shutdown(); if (streamState != null) { @@ -613,18 +710,31 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ ensureNotClosed(); switch (level) { + case Connection.TRANSACTION_NONE: + break; case Connection.TRANSACTION_READ_UNCOMMITTED: + if (isTxAwareQueriesSupported) + throw new SQLException("Requested isolation level not supported by the server: " + level); + + break; + case Connection.TRANSACTION_READ_COMMITTED: case Connection.TRANSACTION_REPEATABLE_READ: case Connection.TRANSACTION_SERIALIZABLE: - case Connection.TRANSACTION_NONE: + if (isTxAwareQueriesSupported && !isolationLevelSupported(level)) + throw new SQLException("Requested isolation level not supported by the server: " + level); + break; default: throw new SQLException("Invalid transaction isolation level.", SqlStateCode.INVALID_TRANSACTION_LEVEL); } - txIsolation = level; + if (txIsolation != level) { + txIsolation = level; + + updateTransactionParameters(); + } } /** {@inheritDoc} */ @@ -667,6 +777,9 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ if (holdability != HOLD_CURSORS_OVER_COMMIT && holdability != CLOSE_CURSORS_AT_COMMIT) throw new SQLException("Invalid result set holdability value."); + if (isTxAwareQueriesSupported && holdability == HOLD_CURSORS_OVER_COMMIT) + throw new SQLException(HOLDABILITY_ERR_MSG); + this.holdability = holdability; } @@ -898,6 +1011,33 @@ private void checkCursorOptions(int resSetType, int resSetConcurrency) throws SQ return netTimeout; } + /** Adds statement to transaction context. */ + void addToTransaction(JdbcThinTcpIo txIo, int txId, JdbcThinStatement stmt) throws SQLException { + if (!txEnabledForConnection()) + return; + + assert !autoCommit; + assert txId != NONE_TX; + + if (txCtx == null) + txCtx = new TxContext(txIo, txId); + // Check same context returned from server when transaction exists, already. + else if (txCtx.txId != txId || !Objects.equals(txCtx.txIo.nodeId(), txIo.nodeId())) { + throw new IllegalStateException("Nested transactions not supported [" + + "txCtx.txId=" + txId + + ", txCtx.nodeId=" + txCtx.txIo.nodeId() + + ", new.txId=" + txId + + ", new.nodeId=" + txIo.nodeId() + ']'); + } + + txCtx.add(stmt); + } + + /** @return Current transaction id. */ + public int txId() { + return txCtx == null ? NONE_TX : txCtx.txId; + } + /** * Ensures that connection is not closed. * @@ -984,8 +1124,6 @@ JdbcResultWithIo sendRequest(JdbcRequest req, JdbcThinStatement stmt, @Nullable JdbcResponse res = cliIo.sendRequest(req, stmt); - txIo = res.activeTransaction() ? cliIo : null; - if (res.status() == IgniteQueryErrorCode.QUERY_CANCELED && stmt != null && stmt.requestTimeout() != NO_TIMEOUT && reqTimeoutTask != null && reqTimeoutTask.expired.get()) { @@ -1376,6 +1514,9 @@ private void onDisconnect(JdbcThinTcpIo cliIo) { void closeStatement(JdbcThinStatement stmt) { synchronized (stmtsMux) { stmts.remove(stmt); + + if (txCtx != null) + txCtx.remove(stmt); } } @@ -1626,8 +1767,8 @@ private JdbcThinTcpIo cliIo(List nodeIds) { if (!partitionAwareness) return singleIo; - if (txIo != null) - return txIo; + if (txCtx != null) + return txCtx.txIo; if (nodeIds == null || nodeIds.isEmpty()) return randomIo(); @@ -1745,6 +1886,8 @@ private void connectInCommonMode() throws SQLException { connCnt.incrementAndGet(); + isTxAwareQueriesSupported = singleIo.isTxAwareQueriesSupported(); + return; } catch (Exception exception) { @@ -1878,6 +2021,8 @@ private IgniteProductVersion connectInBestEffortAffinityMode( handleConnectExceptions(exceptions); + isTxAwareQueriesSupported = defaultIo().isTxAwareQueriesSupported(); + return null; } @@ -1962,6 +2107,12 @@ private int calculateRetryAttemptsCount(JdbcThinTcpIo stickyIo, JdbcRequest req) return NO_RETRIES; } + /** */ + private void logTransactionWarning() { + if (!isTxAwareQueriesSupported) + LOG.warning("Transactions are not supported."); + } + /** * Request Timeout Task */ @@ -2464,4 +2615,72 @@ boolean handleResult(long reqId, JdbcResult res) { return handled; } } + + /** Transaction context. */ + private class TxContext { + /** IO to transaction coordinator. */ + final JdbcThinTcpIo txIo; + + /** Transaction id. */ + final int txId; + + /** Closed flag. */ + boolean closed; + + /** Tracked statements to close results on transaction end. */ + private final Set stmts = Collections.newSetFromMap(new IdentityHashMap<>()); + + /** */ + public TxContext(JdbcThinTcpIo txIo, int txId) { + assert !autoCommit; + + this.txIo = txIo; + this.txId = txId; + } + + /** */ + public void end(boolean commit) throws SQLException { + if (closed) + return; + + closed = true; + + for (JdbcThinStatement stmt : stmts) + stmt.closeResults(); + + sendRequest(new JdbcTxEndRequest(txId, commit), null, null); + } + + /** */ + public void add(JdbcThinStatement stmt) throws SQLException { + if (closed) + throw new SQLException("Transaction context closed"); + + stmts.add(stmt); + } + + /** */ + public void remove(JdbcThinStatement stmt) { + if (closed) + return; + + stmts.remove(stmt); + } + } + + /** */ + public static TransactionIsolation isolation(int jdbcIsolation) throws SQLException { + switch (jdbcIsolation) { + case TRANSACTION_READ_COMMITTED: + return TransactionIsolation.READ_COMMITTED; + case TRANSACTION_REPEATABLE_READ: + return TransactionIsolation.REPEATABLE_READ; + case TRANSACTION_SERIALIZABLE: + return TransactionIsolation.SERIALIZABLE; + case TRANSACTION_NONE: + case TRANSACTION_READ_UNCOMMITTED: + default: + throw new SQLException("Transaction level not supported by the server: " + jdbcIsolation); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java index 22644053c30c5..3758e9625a3ec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java @@ -46,7 +46,6 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcTableMeta; import org.apache.ignite.internal.processors.query.schema.management.IndexDescriptor; -import static java.sql.Connection.TRANSACTION_NONE; import static java.sql.ResultSet.CONCUR_READ_ONLY; import static java.sql.ResultSet.HOLD_CURSORS_OVER_COMMIT; import static java.sql.ResultSet.TYPE_FORWARD_ONLY; @@ -639,17 +638,17 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData { /** {@inheritDoc} */ @Override public int getDefaultTransactionIsolation() throws SQLException { - return TRANSACTION_NONE; + return conn.defaultTransactionIsolation(); } /** {@inheritDoc} */ @Override public boolean supportsTransactions() throws SQLException { - return false; + return conn.isTxAwareQueriesSupported; } /** {@inheritDoc} */ @Override public boolean supportsTransactionIsolationLevel(int level) throws SQLException { - return false; + return conn.isTxAwareQueriesSupported && conn.isolationLevelSupported(level); } /** {@inheritDoc} */ @@ -659,7 +658,7 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData { /** {@inheritDoc} */ @Override public boolean supportsDataManipulationTransactionsOnly() throws SQLException { - return false; + return conn.isTxAwareQueriesSupported; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java index 13ae9b596d664..6795d0db461eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinStatement.java @@ -228,8 +228,19 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg return; } - JdbcQueryExecuteRequest req = new JdbcQueryExecuteRequest(stmtType, schema, pageSize, - maxRows, conn.getAutoCommit(), explicitTimeout, sql, args == null ? null : args.toArray(new Object[args.size()])); + boolean autoCommit = conn.getAutoCommit(); + + JdbcQueryExecuteRequest req = new JdbcQueryExecuteRequest( + stmtType, + schema, + pageSize, + maxRows, + autoCommit, + explicitTimeout, + sql, + args == null ? null : args.toArray(new Object[args.size()]), + conn.txId() + ); JdbcResultWithIo resWithIo = conn.sendRequest(req, this, null); @@ -245,6 +256,9 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg if (res0 instanceof JdbcQueryExecuteResult) { JdbcQueryExecuteResult res = (JdbcQueryExecuteResult)res0; + if (!autoCommit) + conn.addToTransaction(stickyIo, res.txId(), this); + resultSets = Collections.singletonList(new JdbcThinResultSet(this, res.cursorId(), pageSize, res.last(), res.items(), res.isQuery(), conn.autoCloseServerCursor(), res.updateCount(), closeOnCompletion, stickyIo)); @@ -252,6 +266,9 @@ protected void execute0(JdbcStatementType stmtType, String sql, List arg else if (res0 instanceof JdbcQueryExecuteMultipleStatementsResult) { JdbcQueryExecuteMultipleStatementsResult res = (JdbcQueryExecuteMultipleStatementsResult)res0; + if (!autoCommit) + conn.addToTransaction(stickyIo, res.txId(), this); + List resInfos = res.results(); resultSets = new ArrayList<>(resInfos.size()); @@ -409,7 +426,7 @@ private JdbcResult sendFile(JdbcBulkLoadAckResult cmdRes, JdbcThinTcpIo stickyIO * Close results. * @throws SQLException On error. */ - private void closeResults() throws SQLException { + void closeResults() throws SQLException { if (resultSets != null) { for (JdbcThinResultSet rs : resultSets) rs.close0(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java index 7ffc1cadbd6cd..14ad0d7f18f8a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinTcpIo.java @@ -23,13 +23,16 @@ import java.net.InetSocketAddress; import java.net.Socket; import java.sql.SQLException; +import java.util.Arrays; import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Random; +import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; import javax.cache.configuration.Factory; @@ -67,8 +70,10 @@ import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.transactions.TransactionIsolation; import static java.lang.Math.abs; +import static org.apache.ignite.internal.jdbc.thin.JdbcThinConnection.isolation; import static org.apache.ignite.internal.jdbc.thin.JdbcThinUtils.nullableBooleanToByte; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.DEFAULT_NESTED_TX_MODE; @@ -103,8 +108,11 @@ public class JdbcThinTcpIo { /** Version 2.13.0. */ private static final ClientListenerProtocolVersion VER_2_13_0 = ClientListenerProtocolVersion.create(2, 13, 0); + /** Version 2.17.0. */ + private static final ClientListenerProtocolVersion VER_2_17_0 = ClientListenerProtocolVersion.create(2, 17, 0); + /** Current version. */ - private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_13_0; + private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_17_0; /** Initial output stream capacity for handshake. */ private static final int HANDSHAKE_MSG_SIZE = 13; @@ -158,7 +166,13 @@ public class JdbcThinTcpIo { private final ClientListenerProtocolVersion srvProtoVer; /** Protocol context (version, supported features, etc). */ - private JdbcProtocolContext protoCtx; + private final JdbcProtocolContext protoCtx; + + /** + * Transaction modes supported by the server. + * @see org.apache.ignite.configuration.TransactionConfiguration#TX_AWARE_QUERIES_SUPPORTED_MODES + */ + private final Set isolationLevelsSupported; /** Binary context for serialization/deserialization of binary objects. */ private final BinaryContext ctx; @@ -246,6 +260,8 @@ else if (ConnectionProperties.SSL_MODE_DISABLE.equalsIgnoreCase(connProps.getSsl srvProtoVer = handshakeRes.serverProtocolVersion(); + isolationLevelsSupported = handshakeRes.isolationLevelsSupported(); + protoCtx = new JdbcProtocolContext(srvProtoVer, handshakeRes.features(), connProps.isKeepBinary()); } @@ -319,6 +335,13 @@ private HandshakeResult handshake(ClientListenerProtocolVersion ver) throws IOEx if (ver.compareTo(VER_2_13_0) >= 0) writer.writeString(connProps.getQueryEngine()); + if (ver.compareTo(VER_2_17_0) >= 0) { + writer.writeByte(connProps.getTransactionConcurrency().ordinal()); + writer.writeByte(isolation(JdbcThinConnection.DFLT_ISOLATION).ordinal()); + writer.writeInt(connProps.getTransactionTimeout()); + writer.writeString(connProps.getTransactionLabel()); + } + if (!F.isEmpty(connProps.getUsername())) { assert ver.compareTo(VER_2_5_0) >= 0 : "Authentication is supported since 2.5"; @@ -357,6 +380,12 @@ private HandshakeResult handshake(ClientListenerProtocolVersion ver) throws IOEx EnumSet features = JdbcThinFeature.enumSet(srvFeatures); handshakeRes.features(features); + + if (features.contains(JdbcThinFeature.TX_AWARE_QUERIES)) { + handshakeRes.isolationLevelsSupported(Arrays.stream(reader.readIntArray()) + .mapToObj(TransactionIsolation::fromOrdinal) + .collect(Collectors.toSet())); + } } } else { @@ -383,7 +412,8 @@ private HandshakeResult handshake(ClientListenerProtocolVersion ver) throws IOEx + ", url=" + connProps.getUrl() + " address=" + sockAddr + ']', SqlStateCode.CONNECTION_REJECTED); } - if (VER_2_9_0.equals(srvProtoVer0) + if (VER_2_13_0.equals(srvProtoVer0) + || VER_2_9_0.equals(srvProtoVer0) || VER_2_8_0.equals(srvProtoVer0) || VER_2_7_0.equals(srvProtoVer0) || VER_2_5_0.equals(srvProtoVer0) @@ -691,6 +721,23 @@ boolean isCustomObjectSupported() { return protoCtx.isFeatureSupported(JdbcThinFeature.CUSTOM_OBJECT); } + /** + * Whether SQL transactions are supported by the server or not. + * + * @return {@code true} if SQL transactions supported, {@code false} otherwise. + */ + boolean isTxAwareQueriesSupported() { + return protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES); + } + + /** + * @param isolation Transaction isolation level. + * @return {@code True} if transaction isolation mode supported by the server, {@code false} otherwise. + */ + boolean isIsolationLevelSupported(TransactionIsolation isolation) { + return isolationLevelsSupported.contains(isolation); + } + /** * Get next server index. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java index b17248166fa84..9f15ba184fb88 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ValidationOnNodeJoinUtils.java @@ -73,6 +73,7 @@ import static org.apache.ignite.configuration.DeploymentMode.ISOLATED; import static org.apache.ignite.configuration.DeploymentMode.PRIVATE; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED; +import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_AWARE_QUERIES_ENABLED; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_SERIALIZABLE_ENABLED; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isDefaultDataRegionPersistent; import static org.apache.ignite.internal.processors.security.SecurityUtils.nodeSecurityContext; @@ -575,6 +576,18 @@ private static void checkTransactionConfiguration( ", locTxSerializableEnabled=" + locTxCfg.isTxSerializableEnabled() + ", rmtTxSerializableEnabled=" + rmtTxSer + ']'); } + + Boolean rmtTxAwareQryEnabled = rmt.attribute(ATTR_TX_AWARE_QUERIES_ENABLED); + + if (rmtTxAwareQryEnabled != null) { + TransactionConfiguration locTxCfg = ctx.config().getTransactionConfiguration(); + + if (!rmtTxAwareQryEnabled.equals(locTxCfg.isTxAwareQueriesEnabled())) + throw new IgniteCheckedException("Transactions aware queries enabled mismatch " + + "(fix txAwareQueriesEnabled property) [rmtNodeId=" + rmt.id() + + ", locTxAwareQueriesEnabled=" + locTxCfg.isTxAwareQueriesEnabled() + + ", rmtTxAwareQueriesEnabled=" + rmtTxAwareQryEnabled + ']'); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java index aae72871fe252..5c5f1929e6d63 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/tree/BPlusTree.java @@ -5931,6 +5931,16 @@ public interface TreeRowClosure { */ public boolean apply(BPlusTree tree, BPlusIO io, long pageAddr, int idx) throws IgniteCheckedException; + + /** + * Optional operation to provide access to the last analyzed row. + * Can be used to optimize chained filters and omit several tree reads. + * + * @return Last row that was analyzed or {@code null}. + */ + public default L lastRow() { + return null; + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index 5c30d4a5e0445..e0f7d1c73894f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -998,7 +998,7 @@ public IgniteInternalTx tx(IgniteInternalTx tx) { /** * @return Transaction for current thread. */ - public T tx() { + public T tx() { IgniteInternalTx tx = txContext(); return tx != null ? (T)tx : (T)tx(null, Thread.currentThread().getId()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java index fa7d4efa07cbb..3cdbf56b82eaf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerAbstractConnectionContext.java @@ -20,9 +20,11 @@ import java.util.Collections; import java.util.Map; import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException; +import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; import org.apache.ignite.internal.processors.security.SecurityContext; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; @@ -36,6 +38,9 @@ * Base connection context. */ public abstract class ClientListenerAbstractConnectionContext implements ClientListenerConnectionContext { + /** Transaciton id that mean no transaction. */ + public static final int NONE_TX = 0; + /** Kernal context. */ protected final GridKernalContext ctx; @@ -61,6 +66,9 @@ public abstract class ClientListenerAbstractConnectionContext implements ClientL */ private String clientDesc; + /** Tx id. */ + private final AtomicInteger txIdSeq = new AtomicInteger(); + /** * Constructor. * @@ -123,6 +131,8 @@ protected void authenticate(GridNioSession ses, String user, String pwd) throws /** {@inheritDoc} */ @Override public void onDisconnected() { + cleanupTxs(); + if (ctx.security().enabled()) ctx.security().onSessionExpired(secCtx.subject().id()); } @@ -149,6 +159,39 @@ public String clientDescriptor() { return clientDesc; } + /** + * Next transaction id for this connection. + */ + public int nextTxId() { + int txId = txIdSeq.incrementAndGet(); + + return txId == NONE_TX ? txIdSeq.incrementAndGet() : txId; + } + + /** + * Transaction context by transaction id. + * + * @param txId Tx ID. + */ + public abstract @Nullable ClientTxContext txContext(int txId); + + /** + * Add new transaction context to connection. + * + * @param txCtx Tx context. + */ + public abstract void addTxContext(ClientTxContext txCtx); + + /** + * Remove transaction context from connection. + * + * @param txId Tx ID. + */ + public abstract void removeTxContext(int txId); + + /** */ + protected abstract void cleanupTxs(); + /** {@inheritDoc} */ @Override public Map attributes() { return F.isEmpty(userAttrs) ? Collections.emptyMap() : Collections.unmodifiableMap(userAttrs); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientTxSupport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientTxSupport.java new file mode 100644 index 0000000000000..df3939679e623 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientTxSupport.java @@ -0,0 +1,148 @@ +/* + * 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.processors.odbc; + +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; + +/** */ +public interface ClientTxSupport { + /** + * Starts new client transaction. + * + * @param ctx Client connection context. + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @param timeout Transaction timeout. + * @param lb Transaction label. + * @return Transaction id. + */ + default int startClientTransaction( + ClientListenerAbstractConnectionContext ctx, + TransactionConcurrency concurrency, + TransactionIsolation isolation, + long timeout, + String lb + ) { + GridNearTxLocal tx; + + ctx.kernalContext().gateway().readLock(); + + try { + tx = ctx.kernalContext().cache().context().tm().newTx( + false, + false, + null, + concurrency, + isolation, + timeout, + true, + 0, + lb + ); + } + finally { + ctx.kernalContext().gateway().readUnlock(); + } + + try { + tx.suspend(); + + int txId = ctx.nextTxId(); + + ctx.addTxContext(new ClientTxContext(txId, tx)); + + return txId; + } + catch (Exception e) { + try { + tx.close(); + } + catch (Exception e1) { + e.addSuppressed(e1); + } + + throw startTxException(e); + } + } + + /** + * End transaction asynchronously. + * @param ctx Client connection context. + * @param txId Transaction id. + * @param committed If {@code true} transaction must be committed, rollback otherwise. + */ + default IgniteInternalFuture endTxAsync(ClientListenerAbstractConnectionContext ctx, int txId, boolean committed) { + ClientTxContext txCtx = ctx.txContext(txId); + + if (txCtx == null && !committed) + return new GridFinishedFuture<>(); + + if (txCtx == null) + throw transactionNotFoundException(); + + try { + txCtx.acquire(committed); + + if (committed) + return txCtx.tx().context().commitTxAsync(txCtx.tx()); + else + return txCtx.tx().rollbackAsync(); + } + catch (IgniteCheckedException e) { + throw endTxException(e); + } + finally { + ctx.removeTxContext(txId); + + try { + txCtx.release(false); + } + catch (IgniteCheckedException ignore) { + // No-op. + } + } + } + + /** + * @param cause Exception cause. + * @return Protocol specific start transaction exception. + */ + default RuntimeException startTxException(Exception cause) { + return new UnsupportedOperationException(); + } + + /** + * @param cause Exception cause. + * @return Protocol specific end transaction exception. + */ + default RuntimeException endTxException(IgniteCheckedException cause) { + return new UnsupportedOperationException(); + } + + /** @return Protocol specific transaction not found exception. */ + default RuntimeException transactionNotFoundException() { + return new UnsupportedOperationException(); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java index 66721e2865aba..935e209151074 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java @@ -27,16 +27,22 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.binary.BinaryReaderExImpl; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext; import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser; import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.ClientListenerResponseSender; +import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; +import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryEngineConfigurationEx; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.nio.GridNioSession; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.jdbc.thin.JdbcThinUtils.nullableBooleanFromByte; import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.JDBC_CLIENT; @@ -72,8 +78,11 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte /** Version 2.13.0: adds choose of query engine support. */ static final ClientListenerProtocolVersion VER_2_13_0 = ClientListenerProtocolVersion.create(2, 13, 0); + /** Version 2.17.0: adds transaction default parameters. */ + static final ClientListenerProtocolVersion VER_2_17_0 = ClientListenerProtocolVersion.create(2, 17, 0); + /** Current version. */ - public static final ClientListenerProtocolVersion CURRENT_VER = VER_2_13_0; + public static final ClientListenerProtocolVersion CURRENT_VER = VER_2_17_0; /** Supported versions. */ private static final Set SUPPORTED_VERS = new HashSet<>(); @@ -102,8 +111,12 @@ public class JdbcConnectionContext extends ClientListenerAbstractConnectionConte /** Last reported affinity topology version. */ private AtomicReference lastAffinityTopVer = new AtomicReference<>(); + /** Transaction context. */ + private @Nullable ClientTxContext txCtx; + static { SUPPORTED_VERS.add(CURRENT_VER); + SUPPORTED_VERS.add(VER_2_13_0); SUPPORTED_VERS.add(VER_2_9_0); SUPPORTED_VERS.add(VER_2_8_0); SUPPORTED_VERS.add(VER_2_7_0); @@ -192,6 +205,9 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin byte[] cliFeatures = reader.readByteArray(); features = JdbcThinFeature.enumSet(cliFeatures); + + if (!ctx.config().getTransactionConfiguration().isTxAwareQueriesEnabled()) + features.remove(JdbcThinFeature.TX_AWARE_QUERIES); } if (ver.compareTo(VER_2_13_0) >= 0) { @@ -216,6 +232,18 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin } } + TransactionConcurrency concurrency = null; + TransactionIsolation isolation = null; + int timeout = 0; + String lb = null; + + if (ver.compareTo(VER_2_17_0) >= 0) { + concurrency = TransactionConcurrency.fromOrdinal(reader.readByte()); + isolation = TransactionIsolation.fromOrdinal(reader.readByte()); + timeout = reader.readInt(); + lb = reader.readString(); + } + if (ver.compareTo(VER_2_5_0) >= 0) { String user = null; String passwd = null; @@ -252,7 +280,9 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin handler = new JdbcRequestHandler(busyLock, snd, maxCursors, distributedJoins, enforceJoinOrder, collocated, replicatedOnly, autoCloseCursors, lazyExec, skipReducerOnUpdate, qryEngine, - dataPageScanEnabled, updateBatchSize, ver, this); + dataPageScanEnabled, updateBatchSize, + concurrency, isolation, timeout, lb, + ver, this); handler.start(); } @@ -274,6 +304,44 @@ public JdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin super.onDisconnected(); } + /** {@inheritDoc} */ + @Override public @Nullable ClientTxContext txContext(int txId) { + ensureSameTransaction(txId); + + return txCtx; + } + + /** {@inheritDoc} */ + @Override public void addTxContext(ClientTxContext txCtx) { + if (this.txCtx != null) + throw new IgniteSQLException("Too many transactions", IgniteQueryErrorCode.QUERY_CANCELED); + + this.txCtx = txCtx; + } + + /** {@inheritDoc} */ + @Override public void removeTxContext(int txId) { + ensureSameTransaction(txId); + + txCtx = null; + } + + /** */ + private void ensureSameTransaction(int txId) { + if (txCtx != null && txCtx.txId() != txId) { + throw new IllegalStateException("Unknown transaction " + + "[serverTxId=" + (txCtx == null ? null : txCtx.txId()) + ", txId=" + txId + ']'); + } + } + + /** {@inheritDoc} */ + @Override protected void cleanupTxs() { + if (txCtx != null) + txCtx.close(); + + txCtx = null; + } + /** * @return Retrieves current affinity topology version and sets it as a last if it was changed, false otherwise. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java index 37180342a42cf..121c16c747949 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteMultipleStatementsResult.java @@ -38,6 +38,9 @@ public class JdbcQueryExecuteMultipleStatementsResult extends JdbcResult { /** Flag indicating the query has no unfetched results for the first query. */ private boolean last; + /** Transaction id. */ + private int txId; + /** * Default constructor. */ @@ -51,11 +54,12 @@ public class JdbcQueryExecuteMultipleStatementsResult extends JdbcResult { * @param last Flag indicating the query has no unfetched results for the first query. */ public JdbcQueryExecuteMultipleStatementsResult(List results, - List> items, boolean last) { + List> items, boolean last, int txId) { super(QRY_EXEC_MULT); this.results = results; this.items = items; this.last = last; + this.txId = txId; } /** @@ -79,6 +83,13 @@ public boolean isLast() { return last; } + /** + * @return Transaction id. + */ + public int txId() { + return txId; + } + /** {@inheritDoc} */ @Override public void writeBinary( BinaryWriterExImpl writer, @@ -100,6 +111,9 @@ public boolean isLast() { } else writer.writeInt(0); + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + writer.writeInt(txId); } @@ -131,6 +145,9 @@ public boolean isLast() { items = JdbcUtils.readItems(reader, protoCtx); } } + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + txId = reader.readInt(); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java index bace9086104f3..52a839071d1d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteRequest.java @@ -60,6 +60,9 @@ public class JdbcQueryExecuteRequest extends JdbcRequest { /** Explicit timeout. */ private boolean explicitTimeout; + /** Transaction id. */ + private int txId; + /** */ JdbcQueryExecuteRequest() { super(QRY_EXEC); @@ -75,9 +78,10 @@ public class JdbcQueryExecuteRequest extends JdbcRequest { * @param autoCommit Connection auto commit flag state. * @param sqlQry SQL query. * @param args Arguments list. + * @param txId Transaction id. */ public JdbcQueryExecuteRequest(JdbcStatementType stmtType, String schemaName, int pageSize, int maxRows, - boolean autoCommit, boolean explicitTimeout, String sqlQry, Object[] args) { + boolean autoCommit, boolean explicitTimeout, String sqlQry, Object[] args, int txId) { super(QRY_EXEC); this.schemaName = F.isEmpty(schemaName) ? null : schemaName; @@ -88,6 +92,7 @@ public JdbcQueryExecuteRequest(JdbcStatementType stmtType, String schemaName, in this.stmtType = stmtType; this.autoCommit = autoCommit; this.explicitTimeout = explicitTimeout; + this.txId = txId; } /** @@ -166,8 +171,11 @@ boolean autoCommit() { if (protoCtx.isAffinityAwarenessSupported()) writer.writeBoolean(partResReq); - if (protoCtx.features().contains(JdbcThinFeature.QUERY_TIMEOUT)) + if (protoCtx.isFeatureSupported(JdbcThinFeature.QUERY_TIMEOUT)) writer.writeBoolean(explicitTimeout); + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + writer.writeInt(txId); } /** {@inheritDoc} */ @@ -205,8 +213,11 @@ boolean autoCommit() { if (protoCtx.isAffinityAwarenessSupported()) partResReq = reader.readBoolean(); - if (protoCtx.features().contains(JdbcThinFeature.QUERY_TIMEOUT)) + if (protoCtx.isFeatureSupported(JdbcThinFeature.QUERY_TIMEOUT)) explicitTimeout = reader.readBoolean(); + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + txId = reader.readInt(); } /** @@ -231,6 +242,11 @@ public boolean explicitTimeout() { return explicitTimeout; } + /** @return Transaction id. */ + public int txId() { + return txId; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(JdbcQueryExecuteRequest.class, this, super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java index cb6e847fdd853..293310c69ec5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcQueryExecuteResult.java @@ -47,6 +47,9 @@ public class JdbcQueryExecuteResult extends JdbcResult { /** Partition result. */ private PartitionResult partRes; + /** Transaction id. */ + private int txId; + /** * Constructor. */ @@ -59,8 +62,9 @@ public class JdbcQueryExecuteResult extends JdbcResult { * @param items Query result rows. * @param last Flag indicates the query has no unfetched results. * @param partRes partition result to use for best affort affinity on the client side. + * @param txId Transaction id. */ - JdbcQueryExecuteResult(long cursorId, List> items, boolean last, PartitionResult partRes) { + JdbcQueryExecuteResult(long cursorId, List> items, boolean last, PartitionResult partRes, int txId) { super(QRY_EXEC); this.cursorId = cursorId; @@ -68,14 +72,16 @@ public class JdbcQueryExecuteResult extends JdbcResult { this.last = last; isQuery = true; this.partRes = partRes; + this.txId = txId; } /** * @param cursorId Cursor ID. * @param updateCnt Update count for DML queries. * @param partRes partition result to use for best affort affinity on the client side. + * @param txId Transaction id. */ - public JdbcQueryExecuteResult(long cursorId, long updateCnt, PartitionResult partRes) { + public JdbcQueryExecuteResult(long cursorId, long updateCnt, PartitionResult partRes, int txId) { super(QRY_EXEC); this.cursorId = cursorId; @@ -83,6 +89,7 @@ public JdbcQueryExecuteResult(long cursorId, long updateCnt, PartitionResult par isQuery = false; this.updateCnt = updateCnt; this.partRes = partRes; + this.txId = txId; } /** @@ -120,6 +127,13 @@ public long updateCount() { return updateCnt; } + /** + * @return Transaction id. + */ + public int txId() { + return txId; + } + /** {@inheritDoc} */ @Override public void writeBinary( BinaryWriterExImpl writer, @@ -144,6 +158,9 @@ public long updateCount() { if (protoCtx.isAffinityAwarenessSupported() && partRes != null) PartitionResultMarshaler.marshal(writer, partRes); + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + writer.writeInt(txId); } /** {@inheritDoc} */ @@ -169,6 +186,9 @@ public long updateCount() { if (protoCtx.isAffinityAwarenessSupported() && reader.readBoolean()) partRes = PartitionResultMarshaler.unmarshal(reader); + + if (protoCtx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) + txId = reader.readInt(); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java index 34c6882f54f7d..285a36912bb21 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequest.java @@ -88,6 +88,12 @@ public class JdbcRequest extends ClientListenerRequestNoId implements JdbcRawBin /** Update binary type name request. */ public static final byte BINARY_TYPE_NAME_PUT = 20; + /** Sets transaction parameters request. */ + public static final byte TX_SET_PARAMS = 21; + + /** Finish transaction request. */ + public static final byte TX_END = 22; + /** Request Id generator. */ private static final AtomicLong REQ_ID_GENERATOR = new AtomicLong(); @@ -249,6 +255,16 @@ public static JdbcRequest readRequest( break; + case TX_SET_PARAMS: + req = new JdbcSetTxParametersRequest(); + + break; + + case TX_END: + req = new JdbcTxEndRequest(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + reqType + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java index e3f6347fe3414..bbcd750f587e1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java @@ -32,11 +32,15 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import javax.cache.configuration.Factory; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cache.query.BulkLoadContextCursor; import org.apache.ignite.cache.query.FieldsQueryCursor; import org.apache.ignite.cache.query.QueryCancelledException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.TransactionConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.ThinProtocolFeature; @@ -55,13 +59,16 @@ import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode; import org.apache.ignite.internal.processors.cache.query.QueryCursorEx; import org.apache.ignite.internal.processors.cache.query.SqlFieldsQueryEx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion; import org.apache.ignite.internal.processors.odbc.ClientListenerRequest; import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.ClientListenerResponseSender; +import org.apache.ignite.internal.processors.odbc.ClientTxSupport; import org.apache.ignite.internal.processors.odbc.SqlListenerUtils; import org.apache.ignite.internal.processors.odbc.SqlStateCode; +import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; import org.apache.ignite.internal.processors.query.GridQueryCancel; import org.apache.ignite.internal.processors.query.IgniteSQLException; import org.apache.ignite.internal.processors.query.QueryUtils; @@ -75,8 +82,11 @@ import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.marshaller.MarshallerContext; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.odbc.ClientListenerAbstractConnectionContext.NONE_TX; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_CONTINUE; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_EOF; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_ERROR; @@ -104,11 +114,13 @@ import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_EXEC; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_FETCH; import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.QRY_META; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.TX_END; +import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.TX_SET_PARAMS; /** * JDBC request handler. */ -public class JdbcRequestHandler implements ClientListenerRequestHandler { +public class JdbcRequestHandler implements ClientListenerRequestHandler, ClientTxSupport { /** Jdbc query cancelled response. */ private static final JdbcResponse JDBC_QUERY_CANCELLED_RESPONSE = new JdbcResponse(IgniteQueryErrorCode.QUERY_CANCELED, QueryCancelledException.ERR_MSG); @@ -173,6 +185,10 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler { * @param qryEngine Name of SQL query engine to use. * @param dataPageScanEnabled Enable scan data page mode. * @param updateBatchSize Size of internal batch for DML queries. + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @param timeout Transaction timeout. + * @param lb Transaction label. * @param protocolVer Protocol version. * @param connCtx Jdbc connection context. */ @@ -190,6 +206,10 @@ public JdbcRequestHandler( @Nullable String qryEngine, @Nullable Boolean dataPageScanEnabled, @Nullable Integer updateBatchSize, + @Nullable TransactionConcurrency concurrency, + @Nullable TransactionIsolation isolation, + long timeout, + @Nullable String lb, ClientListenerProtocolVersion protocolVer, JdbcConnectionContext connCtx ) { @@ -215,7 +235,11 @@ public JdbcRequestHandler( skipReducerOnUpdate, dataPageScanEnabled, updateBatchSize, - qryEngine + qryEngine, + concurrency, + isolation, + timeout, + lb ); this.busyLock = busyLock; @@ -361,14 +385,19 @@ JdbcResponse doHandle(JdbcRequest req) { resp = getBinaryType((JdbcBinaryTypeGetRequest)req); break; + case TX_SET_PARAMS: + resp = setTransactionParameters((JdbcSetTxParametersRequest)req); + break; + + case TX_END: + resp = endTransaction((JdbcTxEndRequest)req); + break; + default: resp = new JdbcResponse(IgniteQueryErrorCode.UNSUPPORTED_OPERATION, "Unsupported JDBC request [req=" + req + ']'); } - if (resp != null) - resp.activeTransaction(connCtx.kernalContext().cache().context().tm().inUserTx()); - return resp; } finally { @@ -462,7 +491,7 @@ private JdbcResponse processBulkLoadFileBatch(JdbcBulkLoadBatchRequest req) { throw new IllegalArgumentException(); } - return resultToResonse(new JdbcQueryExecuteResult(req.cursorId(), proc.updateCnt(), null)); + return resultToResonse(new JdbcQueryExecuteResult(req.cursorId(), proc.updateCnt(), null, NONE_TX)); } catch (Exception e) { U.error(null, "Error processing file batch", e); @@ -502,8 +531,17 @@ private JdbcResponse processBulkLoadFileBatch(JdbcBulkLoadBatchRequest req) { writer.writeUuid(connCtx.kernalContext().localNodeId()); // Write all features supported by the node. - if (protocolVer.compareTo(VER_2_9_0) >= 0) - writer.writeByteArray(ThinProtocolFeature.featuresAsBytes(connCtx.protocolContext().features())); + if (protocolVer.compareTo(VER_2_9_0) >= 0) { + JdbcProtocolContext ctx = connCtx.protocolContext(); + + writer.writeByteArray(ThinProtocolFeature.featuresAsBytes(ctx.features())); + + if (ctx.isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES)) { + writer.writeIntArray(TransactionConfiguration.TX_AWARE_QUERIES_SUPPORTED_MODES.stream() + .mapToInt(TransactionIsolation::ordinal) + .toArray()); + } + } } /** @@ -616,7 +654,11 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { qry.setSchema(schemaName); - List>> results = querySqlFields(qry, cancel); + int txId = txId(req.txId()); + + List>> results = txEnabledForConnection() + ? invokeInTransaction(txId, req.autoCommit(), qry, cancel) + : querySqlFields(qry, cancel); FieldsQueryCursor> fieldsCur = results.get(0); @@ -649,11 +691,15 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { if (fieldsCur instanceof QueryCursorImpl) partRes = ((QueryCursorImpl>)fieldsCur).partitionResult(); - if (cur.isQuery()) - res = new JdbcQueryExecuteResult(cur.cursorId(), cur.fetchRows(), !cur.hasNext(), - isClientPartitionAwarenessApplicable(req.partitionResponseRequest(), partRes) ? - partRes : - null); + if (cur.isQuery()) { + res = new JdbcQueryExecuteResult( + cur.cursorId(), + cur.fetchRows(), + !cur.hasNext(), + isClientPartitionAwarenessApplicable(req.partitionResponseRequest(), partRes) ? partRes : null, + txId + ); + } else { List> items = cur.fetchRows(); @@ -662,10 +708,12 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { "Invalid result set for not-SELECT query. [qry=" + sql + ", res=" + S.toString(List.class, items) + ']'; - res = new JdbcQueryExecuteResult(cur.cursorId(), (Long)items.get(0).get(0), - isClientPartitionAwarenessApplicable(req.partitionResponseRequest(), partRes) ? - partRes : - null); + res = new JdbcQueryExecuteResult( + cur.cursorId(), + (Long)items.get(0).get(0), + isClientPartitionAwarenessApplicable(req.partitionResponseRequest(), partRes) ? partRes : null, + txId + ); } if (res.last() && (!res.isQuery() || autoCloseCursors)) { @@ -708,7 +756,7 @@ private JdbcResponse executeQuery(JdbcQueryExecuteRequest req) { jdbcResults.add(jdbcRes); } - return resultToResonse(new JdbcQueryExecuteMultipleStatementsResult(jdbcResults, items, last)); + return resultToResonse(new JdbcQueryExecuteMultipleStatementsResult(jdbcResults, items, last, txId)); } } catch (Exception e) { @@ -746,6 +794,75 @@ else if (X.cause(e, IgniteSQLException.class) != null) { } } + /** + * Invokes {@code qry} inside a transaction with id equals to {@code txId}. + */ + private List>> invokeInTransaction( + int txId, + boolean autoCommit, + SqlFieldsQueryEx qry, + GridQueryCancel cancel + ) throws IgniteCheckedException { + ClientTxContext txCtx = connCtx.txContext(txId); + + if (txCtx == null) + throw new IgniteException("Transaction not found [txId=" + txId + ']'); + + boolean err = false; + + try { + txCtx.acquire(true); + + return querySqlFields(qry, cancel); + } + catch (Exception e) { + err = true; + + throw e; + } + finally { + try { + txCtx.release(true); + } + catch (Exception e) { + log.warning("Failed to release client transaction context", e); + } + + if (autoCommit) + endTransaction(qry.getTimeout(), txId, !err); + } + } + + /** @return {@code True} if transaction enabled fro connection, {@code false} otherwise. */ + private boolean txEnabledForConnection() { + return connCtx.protocolContext().isFeatureSupported(JdbcThinFeature.TX_AWARE_QUERIES) + && cliCtx.isolation() != null; + } + + /** */ + private int txId(int txId) { + if (txId != NONE_TX || !txEnabledForConnection()) + return txId; + + return startClientTransaction( + connCtx, + cliCtx.concurrency(), + cliCtx.isolation(), + cliCtx.transactionTimeout(), + cliCtx.transactionLabel() + ); + } + + /** */ + private void endTransaction(int timeout, int txId, boolean committed) throws IgniteCheckedException { + IgniteInternalFuture endTxFut = endTxAsync(connCtx, txId, committed); + + if (timeout != -1) + endTxFut.get(timeout); + else + endTxFut.get(); + } + /** */ private List>> querySqlFields(SqlFieldsQueryEx qry, GridQueryCancel cancel) { return connCtx.kernalContext().query().querySqlFields(null, qry, @@ -1316,6 +1433,42 @@ private JdbcResponse getSchemas(JdbcMetaSchemasRequest req) { } } + /** + * Sets transaction parameters for connection. + * + * @param req Request + * @return resulting {@link JdbcResponse}. + */ + private JdbcResponse setTransactionParameters(JdbcSetTxParametersRequest req) { + cliCtx.txParameters( + req.concurrency(), + req.isolation(), + req.timeout(), + req.label() + ); + + return resultToResonse(null); + } + + /** + * End transaction. + * + * @param req Request + * @return resulting {@link JdbcResponse}. + */ + private JdbcResponse endTransaction(JdbcTxEndRequest req) { + try { + endTxAsync(connCtx, req.txId(), req.committed()).get(); + + return resultToResonse(new JdbcTxEndResult(req.requestId())); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to end transaction [reqId=" + req.requestId() + ", req=" + req + ']', e); + + return exceptionToResult(e); + } + } + /** * Create {@link JdbcResponse} bearing appropriate Ignite specific result code if possible * from given {@link Exception}. @@ -1327,6 +1480,21 @@ private JdbcResponse exceptionToResult(Throwable e) { return new JdbcResponse(SqlListenerUtils.exceptionToSqlErrorCode(e), e.getMessage()); } + /** {@inheritDoc} */ + @Override public RuntimeException startTxException(Exception cause) { + return cause instanceof RuntimeException ? (RuntimeException)cause : new IgniteException(cause); + } + + /** {@inheritDoc} */ + @Override public RuntimeException transactionNotFoundException() { + return new IgniteSQLException("Transaction not found", IgniteQueryErrorCode.QUERY_CANCELED); + } + + /** {@inheritDoc} */ + @Override public RuntimeException endTxException(IgniteCheckedException cause) { + return new IgniteSQLException(cause.getMessage(), IgniteQueryErrorCode.QUERY_CANCELED, cause); + } + /** * Ordered batch worker. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java index 4a941a6ffd278..91388fa0094d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResponse.java @@ -34,9 +34,6 @@ public class JdbcResponse extends ClientListenerResponse implements JdbcRawBinar @GridToStringInclude private JdbcResult res; - /** Signals that there is active transactional context. */ - private boolean activeTx; - /** Affinity version. */ private AffinityTopologyVersion affinityVer; @@ -95,20 +92,6 @@ public AffinityTopologyVersion affinityVersion() { return affinityVer; } - /** - * @return True if there's an active transactional on server. - */ - public boolean activeTransaction() { - return activeTx; - } - - /** - * @param activeTx Sets active transaction flag. - */ - public void activeTransaction(boolean activeTx) { - this.activeTx = activeTx; - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(JdbcResponse.class, this, "status", status(), "err", error()); @@ -131,7 +114,7 @@ public void activeTransaction(boolean activeTx) { writer.writeString(error()); if (protoCtx.isAffinityAwarenessSupported()) { - writer.writeBoolean(activeTx); + writer.writeBoolean(false); // Keeping compatibility. writer.writeBoolean(affinityVer != null); @@ -157,7 +140,7 @@ public void activeTransaction(boolean activeTx) { error(reader.readString()); if (protoCtx.isAffinityAwarenessSupported()) { - activeTx = reader.readBoolean(); + reader.readBoolean(); // Keeping compatibility. boolean affVerChanged = reader.readBoolean(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java index b149366de45b9..d5d6519071128 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java @@ -87,6 +87,12 @@ public class JdbcResult implements JdbcRawBinarylizable { /** Get binary type name result. */ static final byte BINARY_TYPE_NAME_GET = 22; + /** Start transaction response. */ + static final byte TX_SET_PARAMS = 23; + + /** End transaction response. */ + static final byte TX_END = 24; + /** Success status. */ private byte type; @@ -229,6 +235,14 @@ public static JdbcResult readResult( break; + case TX_SET_PARAMS: + return null; + + case TX_END: + res = new JdbcTxEndResult(); + + break; + default: throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']'); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcSetTxParametersRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcSetTxParametersRequest.java new file mode 100644 index 0000000000000..c77237dc84661 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcSetTxParametersRequest.java @@ -0,0 +1,122 @@ +/* + * 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.processors.odbc.jdbc; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.jetbrains.annotations.Nullable; + +/** + * JDBC sets transactions parameters for connection. + */ +public class JdbcSetTxParametersRequest extends JdbcRequest { + /** Transaction concurrency control. */ + private TransactionConcurrency concurrency; + + /** Transaction isolation level. */ + private @Nullable TransactionIsolation isolation; + + /** Transaction timeout. */ + private int timeout; + + /** Transaction label. */ + private String lb; + + /** Default constructor is used for deserialization. */ + public JdbcSetTxParametersRequest() { + super(TX_SET_PARAMS); + } + + /** + * @param concurrency Transaction concurrency. + * @param isolation Isolation level. + * @param timeout Timeout. + * @param lb Label. + */ + public JdbcSetTxParametersRequest( + TransactionConcurrency concurrency, + @Nullable TransactionIsolation isolation, + int timeout, + String lb + ) { + this(); + + this.concurrency = concurrency; + this.isolation = isolation; + this.timeout = timeout; + this.lb = lb; + } + + /** {@inheritDoc} */ + @Override public void writeBinary( + BinaryWriterExImpl writer, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.writeBinary(writer, protoCtx); + + writer.writeByte((byte)concurrency.ordinal()); + writer.writeByte((byte)(isolation == null ? -1 : isolation.ordinal())); + writer.writeInt(timeout); + writer.writeString(lb); + } + + /** {@inheritDoc} */ + @Override public void readBinary( + BinaryReaderExImpl reader, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.readBinary(reader, protoCtx); + + concurrency = TransactionConcurrency.fromOrdinal(reader.readByte()); + + byte isolationByte = reader.readByte(); + + isolation = isolationByte == -1 ? null : TransactionIsolation.fromOrdinal(isolationByte); + timeout = reader.readInt(); + lb = reader.readString(); + } + + /** */ + public TransactionConcurrency concurrency() { + return concurrency; + } + + /** */ + public TransactionIsolation isolation() { + return isolation; + } + + /** */ + public long timeout() { + return timeout; + } + + /** */ + public String label() { + return lb; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcSetTxParametersRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcThinFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcThinFeature.java index 7c1a451184e95..a51c4374cd1ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcThinFeature.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcThinFeature.java @@ -33,7 +33,10 @@ public enum JdbcThinFeature implements ThinProtocolFeature { CUSTOM_OBJECT(2), /** Add ability to set explicit query timeout on the cluster node by the JDBC client. */ - QUERY_TIMEOUT(3); + QUERY_TIMEOUT(3), + + /** Transaction aware queries. */ + TX_AWARE_QUERIES(4); /** */ private static final EnumSet ALL_FEATURES_AS_ENUM_SET = EnumSet.allOf(JdbcThinFeature.class); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndRequest.java new file mode 100644 index 0000000000000..4303630582602 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndRequest.java @@ -0,0 +1,84 @@ +/* + * 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.processors.odbc.jdbc; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * JDBC ent transaction request. + */ +public class JdbcTxEndRequest extends JdbcRequest { + /** Transaction id. */ + private int txId; + + /** Flag whether transaction must be committed. */ + private boolean committed; + + /** Default constructor is used for deserialization. */ + public JdbcTxEndRequest() { + super(TX_END); + } + + /** {@inheritDoc} */ + public JdbcTxEndRequest(int txId, boolean committed) { + this(); + + this.txId = txId; + this.committed = committed; + } + + /** @return Transaction id. */ + public int txId() { + return txId; + } + + /** @return {@code true} if transaction was committed on client, {@code false} otherwise. */ + public boolean committed() { + return committed; + } + + /** {@inheritDoc} */ + @Override public void writeBinary( + BinaryWriterExImpl writer, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.writeBinary(writer, protoCtx); + + writer.writeInt(txId); + writer.writeBoolean(committed); + } + + /** {@inheritDoc} */ + @Override public void readBinary( + BinaryReaderExImpl reader, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.readBinary(reader, protoCtx); + + txId = reader.readInt(); + committed = reader.readBoolean(); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcTxEndRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndResult.java new file mode 100644 index 0000000000000..2425922116057 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcTxEndResult.java @@ -0,0 +1,77 @@ +/* + * 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.processors.odbc.jdbc; + +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.internal.binary.BinaryReaderExImpl; +import org.apache.ignite.internal.binary.BinaryWriterExImpl; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * JDBC end transaction result. + */ +public class JdbcTxEndResult extends JdbcResult { + /** ID of initial request. */ + private long reqId; + + /** Default constructor for deserialization purpose. */ + public JdbcTxEndResult() { + super(TX_END); + } + + /** + * @param reqId ID of initial request. + */ + public JdbcTxEndResult(long reqId) { + this(); + + this.reqId = reqId; + } + + /** {@inheritDoc} */ + @Override public void writeBinary( + BinaryWriterExImpl writer, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.writeBinary(writer, protoCtx); + + writer.writeLong(reqId); + } + + /** {@inheritDoc} */ + @Override public void readBinary( + BinaryReaderExImpl reader, + JdbcProtocolContext protoCtx + ) throws BinaryObjectException { + super.readBinary(reader, protoCtx); + + reqId = reader.readLong(); + } + + /** + * @return Request id. + */ + public long reqId() { + return reqId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(JdbcTxEndResult.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java index 8d64ca06fbcd0..2f7240c766302 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcConnectionContext.java @@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.odbc.ClientListenerResponse; import org.apache.ignite.internal.processors.odbc.ClientListenerResponseSender; +import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; import org.apache.ignite.internal.processors.query.QueryEngineConfigurationEx; import org.apache.ignite.internal.util.GridSpinBusyLock; import org.apache.ignite.internal.util.nio.GridNioSession; @@ -225,4 +226,24 @@ public OdbcConnectionContext(GridKernalContext ctx, GridNioSession ses, GridSpin super.onDisconnected(); } + + /** {@inheritDoc} */ + @Override public ClientTxContext txContext(int txId) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void addTxContext(ClientTxContext txCtx) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override public void removeTxContext(int txId) { + throw new UnsupportedOperationException(); + } + + /** {@inheritDoc} */ + @Override protected void cleanupTxs() { + throw new UnsupportedOperationException(); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java index 3e71564137ae8..26a28a33aed79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/odbc/OdbcRequestHandler.java @@ -165,7 +165,11 @@ public OdbcRequestHandler( skipReducerOnUpdate, null, null, - qryEngine + qryEngine, + null, + null, + 0, + null ); this.busyLock = busyLock; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientBitmaskFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientBitmaskFeature.java index ffc966738dd43..d51b8eaaa743e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientBitmaskFeature.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientBitmaskFeature.java @@ -78,7 +78,10 @@ public enum ClientBitmaskFeature implements ThinProtocolFeature { SERVICE_TOPOLOGY(16), /** Cache invoke/invokeAll operations. */ - CACHE_INVOKE(17); + CACHE_INVOKE(17), + + /** Transaction aware queries. */ + TX_AWARE_QUERIES(18); /** */ private static final EnumSet ALL_FEATURES_AS_ENUM_SET = diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java index 135681872667a..5314e1cfdf221 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java @@ -37,6 +37,7 @@ import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler; import org.apache.ignite.internal.processors.platform.client.tx.ClientTxContext; import org.apache.ignite.internal.util.nio.GridNioSession; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.odbc.ClientListenerNioListener.THIN_CLIENT; import static org.apache.ignite.internal.processors.platform.client.ClientBitmaskFeature.USER_ATTRIBUTES; @@ -117,9 +118,6 @@ public class ClientConnectionContext extends ClientListenerAbstractConnectionCon /** Active tx count limit. */ private final int maxActiveTxCnt; - /** Tx id. */ - private final AtomicInteger txIdSeq = new AtomicInteger(); - /** Transactions by transaction id. */ private final Map txs = new ConcurrentHashMap<>(); @@ -196,6 +194,9 @@ public ClientProtocolContext currentProtocolContext() { byte[] cliFeatures = reader.readByteArray(); features = ClientBitmaskFeature.enumSet(cliFeatures); + + if (!ctx.config().getTransactionConfiguration().isTxAwareQueriesEnabled()) + features.remove(ClientBitmaskFeature.TX_AWARE_QUERIES); } currentProtocolContext = new ClientProtocolContext(ver, features); @@ -245,8 +246,6 @@ public ClientProtocolContext currentProtocolContext() { @Override public void onDisconnected() { resReg.clean(); - cleanupTxs(); - super.onDisconnected(); } @@ -295,30 +294,13 @@ public ClientAffinityTopologyVersion checkAffinityTopologyVersion() { } } - /** - * Next transaction id for this connection. - */ - public int nextTxId() { - int txId = txIdSeq.incrementAndGet(); - - return txId == 0 ? txIdSeq.incrementAndGet() : txId; - } - - /** - * Transaction context by transaction id. - * - * @param txId Tx ID. - */ - public ClientTxContext txContext(int txId) { + /** {@inheritDoc} */ + @Override public @Nullable ClientTxContext txContext(int txId) { return txs.get(txId); } - /** - * Add new transaction context to connection. - * - * @param txCtx Tx context. - */ - public void addTxContext(ClientTxContext txCtx) { + /** {@inheritDoc} */ + @Override public void addTxContext(ClientTxContext txCtx) { if (txsCnt.incrementAndGet() > maxActiveTxCnt) { txsCnt.decrementAndGet(); @@ -331,21 +313,15 @@ public void addTxContext(ClientTxContext txCtx) { txs.put(txCtx.txId(), txCtx); } - /** - * Remove transaction context from connection. - * - * @param txId Tx ID. - */ - public void removeTxContext(int txId) { + /** {@inheritDoc} */ + @Override public void removeTxContext(int txId) { txs.remove(txId); txsCnt.decrementAndGet(); } - /** - * - */ - private void cleanupTxs() { + /** {@inheritDoc} */ + @Override public void cleanupTxs() { for (ClientTxContext txCtx : txs.values()) txCtx.close(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxEndRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxEndRequest.java index 8dc070295d865..08a73ff4d6301 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxEndRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxEndRequest.java @@ -20,19 +20,18 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.binary.BinaryRawReader; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; +import org.apache.ignite.internal.processors.odbc.ClientTxSupport; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientRequest; import org.apache.ignite.internal.processors.platform.client.ClientResponse; import org.apache.ignite.internal.processors.platform.client.ClientStatus; import org.apache.ignite.internal.processors.platform.client.IgniteClientException; -import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.lang.GridClosureException; /** * End the transaction request. */ -public class ClientTxEndRequest extends ClientRequest { +public class ClientTxEndRequest extends ClientRequest implements ClientTxSupport { /** Transaction id. */ private final int txId; @@ -58,7 +57,7 @@ public ClientTxEndRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public IgniteInternalFuture processAsync(ClientConnectionContext ctx) { - return endTxAsync(ctx).chain(f -> { + return endTxAsync(ctx, txId, committed).chain(f -> { if (f.error() != null) throw new GridClosureException(f.error()); else @@ -66,36 +65,13 @@ public ClientTxEndRequest(BinaryRawReader reader) { }); } - /** End transaction asynchronously. */ - private IgniteInternalFuture endTxAsync(ClientConnectionContext ctx) { - ClientTxContext txCtx = ctx.txContext(txId); - - if (txCtx == null && !committed) - return new GridFinishedFuture<>(); - - if (txCtx == null) - throw new IgniteClientException(ClientStatus.TX_NOT_FOUND, "Transaction with id " + txId + " not found."); - - try { - txCtx.acquire(committed); - - if (committed) - return txCtx.tx().context().commitTxAsync(txCtx.tx()); - else - return txCtx.tx().rollbackAsync(); - } - catch (IgniteCheckedException e) { - throw new IgniteClientException(ClientStatus.FAILED, e.getMessage(), e); - } - finally { - ctx.removeTxContext(txId); + /** {@inheritDoc} */ + @Override public RuntimeException transactionNotFoundException() { + return new IgniteClientException(ClientStatus.TX_NOT_FOUND, "Transaction with id " + txId + " not found."); + } - try { - txCtx.release(false); - } - catch (IgniteCheckedException ignore) { - // No-op. - } - } + /** {@inheritDoc} */ + @Override public RuntimeException endTxException(IgniteCheckedException cause) { + return new IgniteClientException(ClientStatus.FAILED, cause.getMessage(), cause); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java index be6021580fd0d..d32cc626f9ae6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/tx/ClientTxStartRequest.java @@ -18,7 +18,7 @@ package org.apache.ignite.internal.processors.platform.client.tx; import org.apache.ignite.binary.BinaryRawReader; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; +import org.apache.ignite.internal.processors.odbc.ClientTxSupport; import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext; import org.apache.ignite.internal.processors.platform.client.ClientIntResponse; import org.apache.ignite.internal.processors.platform.client.ClientRequest; @@ -31,7 +31,7 @@ /** * Start transaction request. */ -public class ClientTxStartRequest extends ClientRequest { +public class ClientTxStartRequest extends ClientRequest implements ClientTxSupport { /** Transaction concurrency control. */ private final TransactionConcurrency concurrency; @@ -60,46 +60,12 @@ public ClientTxStartRequest(BinaryRawReader reader) { /** {@inheritDoc} */ @Override public ClientResponse process(ClientConnectionContext ctx) { - GridNearTxLocal tx; - - ctx.kernalContext().gateway().readLock(); - - try { - tx = ctx.kernalContext().cache().context().tm().newTx( - false, - false, - null, - concurrency, - isolation, - timeout, - true, - 0, - lb - ); - } - finally { - ctx.kernalContext().gateway().readUnlock(); - } - - try { - tx.suspend(); - - int txId = ctx.nextTxId(); - - ctx.addTxContext(new ClientTxContext(txId, tx)); - - return new ClientIntResponse(requestId(), txId); - } - catch (Exception e) { - try { - tx.close(); - } - catch (Exception e1) { - e.addSuppressed(e1); - } + return new ClientIntResponse(requestId(), startClientTransaction(ctx, concurrency, isolation, timeout, lb)); + } - throw (e instanceof IgniteClientException) ? (IgniteClientException)e : - new IgniteClientException(ClientStatus.FAILED, e.getMessage(), e); - } + /** {@inheritDoc} */ + @Override public RuntimeException startTxException(Exception e) { + return (e instanceof IgniteClientException) ? (IgniteClientException)e : + new IgniteClientException(ClientStatus.FAILED, e.getMessage(), e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java index f0b1e90f6e3aa..fed8ec75d924f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java @@ -60,6 +60,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.QueryEngineConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.events.CacheQueryExecutedEvent; import org.apache.ignite.internal.GridComponent; import org.apache.ignite.internal.GridKernalContext; @@ -87,6 +88,7 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; import org.apache.ignite.internal.processors.cache.query.CacheQueryFuture; import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType; @@ -303,6 +305,9 @@ public class GridQueryProcessor extends GridProcessorAdapter { /** Global schema SQL views manager. */ private final SchemaSqlViewManager schemaSqlViewMgr; + /** @see TransactionConfiguration#isTxAwareQueriesEnabled() */ + private final boolean txAwareQueriesEnabled; + /** * Constructor. * @@ -344,6 +349,8 @@ public GridQueryProcessor(GridKernalContext ctx) throws IgniteCheckedException { initQueryEngines(); idxBuildStatusStorage = new IndexBuildStatusStorage(ctx); + + txAwareQueriesEnabled = ctx.config().getTransactionConfiguration().isTxAwareQueriesEnabled(); } /** {@inheritDoc} */ @@ -3039,7 +3046,11 @@ public List>> querySqlFields( if (qry.isLocal() && ctx.clientNode()) throw new CacheException("Execution of local SqlFieldsQuery on client node disallowed."); - return executeQuerySafe(cctx, () -> { + final GridNearTxLocal userTx = txAwareQueriesEnabled + ? ctx.cache().context().tm().userTx() + : null; + + return executeQuerySafe(cctx, userTx, () -> { final String schemaName = qry.getSchema() == null ? schemaName(cctx) : qry.getSchema(); IgniteOutClosureX>>> clo = @@ -3058,9 +3069,17 @@ public List>> querySqlFields( failOnMultipleStmts ); + QueryContext qryCtx = QueryContext.of( + qry, + cliCtx, + cancel, + qryProps, + userTx == null ? null : userTx.xidVersion() + ); + if (qry instanceof SqlFieldsQueryEx && ((SqlFieldsQueryEx)qry).isBatched()) { res = qryEngine.queryBatched( - QueryContext.of(qry, cliCtx, cancel, qryProps), + qryCtx, schemaName, qry.getSql(), ((SqlFieldsQueryEx)qry).batchedArguments() @@ -3068,7 +3087,7 @@ public List>> querySqlFields( } else { res = qryEngine.query( - QueryContext.of(qry, cliCtx, cancel, qryProps), + qryCtx, schemaName, qry.getSql(), qry.getArgs() != null ? qry.getArgs() : X.EMPTY_OBJECT_ARRAY @@ -3076,6 +3095,13 @@ public List>> querySqlFields( } } else { + if (userTx != null && txAwareQueriesEnabled) { + throw new IgniteSQLException( + "SQL aware queries are not supported by Indexing query engine", + IgniteQueryErrorCode.UNSUPPORTED_OPERATION + ); + } + res = idx.querySqlFields( schemaName, qry, @@ -3104,7 +3130,7 @@ public List parameterMetaData( ) { checkxModuleEnabled(); - return executeQuerySafe(null, () -> { + return executeQuerySafe(null, null, () -> { final String schemaName = qry.getSchema() == null ? QueryUtils.DFLT_SCHEMA : qry.getSchema(); QueryEngine qryEngine = engineForQuery(cliCtx, qry); @@ -3132,7 +3158,7 @@ public List resultSetMetaData( ) { checkxModuleEnabled(); - return executeQuerySafe(null, () -> { + return executeQuerySafe(null, null, () -> { final String schemaName = qry.getSchema() == null ? QueryUtils.DFLT_SCHEMA : qry.getSchema(); QueryEngine qryEngine = engineForQuery(cliCtx, qry); @@ -3209,27 +3235,43 @@ private QueryEngine engineForQuery(SqlClientContext cliCtx, SqlFieldsQuery qry) * Execute query setting busy lock, preserving current cache context and properly handling checked exceptions. * * @param cctx Cache context. + * @param userTx User transaction. * @param supplier Code to be executed. * @return Result. */ - private T executeQuerySafe(@Nullable final GridCacheContext cctx, GridPlainOutClosure supplier) { + private T executeQuerySafe( + @Nullable final GridCacheContext cctx, + @Nullable final GridNearTxLocal userTx, + GridPlainOutClosure supplier + ) { GridCacheContext oldCctx = curCache.get(); curCache.set(cctx); - if (!busyLock.enterBusy()) - throw new IllegalStateException("Failed to execute query (grid is stopping)."); - try { - return supplier.apply(); + if (!busyLock.enterBusy()) + throw new IllegalStateException("Failed to execute query (grid is stopping)."); + + if (userTx != null) + userTx.suspend(); + + try { + return supplier.apply(); + } + catch (IgniteCheckedException e) { + throw new CacheException(e); + } + finally { + curCache.set(oldCctx); + + busyLock.leaveBusy(); + + if (userTx != null) + userTx.resume(); + } } catch (IgniteCheckedException e) { - throw new CacheException(e); - } - finally { - curCache.set(oldCctx); - - busyLock.leaveBusy(); + throw new IgniteException(e); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java index 7f6a55c20e2cf..e8b581e0bde4a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/SqlClientContext.java @@ -28,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.internal.util.worker.GridWorker; import org.apache.ignite.thread.IgniteThread; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import org.jetbrains.annotations.Nullable; /** @@ -98,6 +100,18 @@ public class SqlClientContext implements AutoCloseable { * the processing the last request. */ private long totalProcessedOrderedReqs; + /** Transaction concurrency control. */ + private TransactionConcurrency concurrency; + + /** Transaction isolation level. */ + private @Nullable TransactionIsolation isolation; + + /** Transaction timeout. */ + private long timeout; + + /** Transaction label. */ + private String lb; + /** Logger. */ private final IgniteLogger log; @@ -113,6 +127,10 @@ public class SqlClientContext implements AutoCloseable { * @param dataPageScanEnabled Enable scan data page mode. * @param updateBatchSize Size of internal batch for DML queries. * @param qryEngine Name of the SQL engine to use. + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @param timeout Transaction timeout. + * @param lb Transaction label. */ public SqlClientContext(GridKernalContext ctx, Factory orderedBatchWorkerFactory, @@ -124,7 +142,11 @@ public SqlClientContext(GridKernalContext ctx, boolean skipReducerOnUpdate, @Nullable Boolean dataPageScanEnabled, @Nullable Integer updateBatchSize, - @Nullable String qryEngine + @Nullable String qryEngine, + @Nullable TransactionConcurrency concurrency, + @Nullable TransactionIsolation isolation, + long timeout, + @Nullable String lb ) { this.ctx = ctx; this.orderedBatchWorkerFactory = orderedBatchWorkerFactory; @@ -137,6 +159,10 @@ public SqlClientContext(GridKernalContext ctx, this.dataPageScanEnabled = dataPageScanEnabled; this.updateBatchSize = updateBatchSize; this.qryEngine = qryEngine; + this.concurrency = concurrency; + this.isolation = isolation; + this.timeout = timeout; + this.lb = lb; log = ctx.log(SqlClientContext.class.getName()); } @@ -338,6 +364,40 @@ public void orderedRequestProcessed() { } } + /** + * Sets transaction parameters. + * @param concurrency Transaction concurrency. + * @param isolation Transaction isolation. + * @param timeout Transaction timeout. + * @param lb Transaction label. + */ + public void txParameters(TransactionConcurrency concurrency, @Nullable TransactionIsolation isolation, long timeout, String lb) { + this.concurrency = concurrency; + this.isolation = isolation; + this.timeout = timeout; + this.lb = lb; + } + + /** */ + public TransactionConcurrency concurrency() { + return concurrency; + } + + /** */ + public @Nullable TransactionIsolation isolation() { + return isolation; + } + + /** */ + public long transactionTimeout() { + return timeout; + } + + /** */ + public String transactionLabel() { + return lb; + } + /** {@inheritDoc} */ @Override public void close() throws Exception { if (streamers == null) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java index 8fb3c4d8a7877..8270636dc07bb 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java @@ -3181,6 +3181,22 @@ public static boolean isSorted(long[] arr) { return true; } + /** + * @param arr Array to check. + * @return {@code True} if array sorted, {@code false} otherwise. + */ + public static boolean isSorted(int[] arr) { + if (isEmpty(arr) || arr.length == 1) + return true; + + for (int i = 1; i < arr.length; i++) { + if (arr[i - 1] > arr[i]) + return false; + } + + return true; + } + /** * Return supplier that suppress any exception throwed by {@code s}. * Returned supplier will produce {@code 0} on any exception in {@code s}. diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TransactionAwareQueriesEnabledTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TransactionAwareQueriesEnabledTest.java new file mode 100644 index 0000000000000..84aee162b570e --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2TransactionAwareQueriesEnabledTest.java @@ -0,0 +1,80 @@ +/* + * 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.processors.cache.index; + +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.processors.query.IgniteSQLException; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.apache.ignite.transactions.Transaction; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; +import org.junit.Test; + +import static org.apache.ignite.testframework.GridTestUtils.assertThrows; + +/** */ +public class H2TransactionAwareQueriesEnabledTest extends GridCommonAbstractTest { + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(true); + cfg.setCacheConfiguration(new CacheConfiguration<>(DEFAULT_CACHE_NAME).setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)); + + return cfg; + } + + /** */ + @Test + public void testFailOnSqlQueryInsideTransaction() throws Exception { + Ignite srv = startGrid(0); + Ignite cli = startClientGrid(1); + + sql(srv, "CREATE TABLE T(ID BIGINT PRIMARY KEY, NAME VARCHAR) WITH \"atomicity=transactional\""); + + awaitPartitionMapExchange(); + + for (Ignite node : new Ignite[]{srv, cli}) { + for (TransactionConcurrency txConcurrenty : TransactionConcurrency.values()) { + for (TransactionIsolation txIsolation : TransactionIsolation.values()) { + try (Transaction tx = node.transactions().txStart(txConcurrenty, txIsolation)) { + assertThrows( + null, + () -> sql(node, "SELECT * FROM T"), + IgniteSQLException.class, + "SQL aware queries are not supported by Indexing query engine" + ); + + tx.rollback(); + } + } + } + } + + } + + /** */ + public List> sql(Ignite node, String sql) { + return node.cache(DEFAULT_CACHE_NAME).query(new SqlFieldsQuery(sql)).getAll(); + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java index 286dd897116c5..78e2896767e59 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java @@ -91,6 +91,7 @@ import org.apache.ignite.internal.processors.cache.index.H2RowCachePageEvictionTest; import org.apache.ignite.internal.processors.cache.index.H2RowCacheSelfTest; import org.apache.ignite.internal.processors.cache.index.H2RowExpireTimeIndexSelfTest; +import org.apache.ignite.internal.processors.cache.index.H2TransactionAwareQueriesEnabledTest; import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest; import org.apache.ignite.internal.processors.cache.index.IndexColumnTypeMismatchTest; import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest; @@ -375,7 +376,9 @@ CacheEventsCdcTest.class, CdcIndexRebuildTest.class, - DumpCacheConfigTest.class + DumpCacheConfigTest.class, + + H2TransactionAwareQueriesEnabledTest.class, }) public class IgniteBinaryCacheQueryTestSuite3 { /** Setup lazy mode default. */