Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,9 @@ public class ExecutionContext<Row> extends AbstractQueryContext implements DataC
/** */
private Object[] correlations = new Object[16];

/** Entries holder per execution thread. */
private static final ThreadLocal<Collection<QueryTxEntry>> txEntriesHolder = new ThreadLocal<>();

/**
* @param qctx Parent base query context.
* @param qryId Query ID.
Expand Down Expand Up @@ -182,7 +185,7 @@ public ExecutionContext(
this.ioTracker = ioTracker;
this.params = params;
this.timeout = timeout;
this.qryTxEntries = qryTxEntries;
this.qryTxEntries = qryTxEntries == null ? txEntriesHolder.get() : qryTxEntries;

startTs = U.currentTimeMillis();

Expand Down Expand Up @@ -421,12 +424,17 @@ public void execute(RunnableX task, Consumer<Throwable> onError) {

executor.execute(qryId, fragmentId(), () -> {
try {
txEntriesHolder.set(qryTxEntries);

if (!isCancelled())
task.run();
}
catch (Throwable e) {
onError.accept(e);
}
finally {
txEntriesHolder.remove();
}
});
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,212 @@
/*
* 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.ArrayList;
import java.util.List;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.Ignition;
import org.apache.ignite.cache.QueryEntity;
import org.apache.ignite.cache.query.SqlFieldsQuery;
import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
import org.apache.ignite.calcite.CalciteQueryEngineConfiguration;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.WithSystemProperty;
import org.apache.ignite.transactions.Transaction;
import org.junit.Test;

import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.internal.processors.query.calcite.CalciteQueryProcessor.IGNITE_CALCITE_USE_QUERY_BLOCKING_TASK_EXECUTOR;
import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertThat;

/**
* Integration test for user defined functions with tx aware.
*/
@WithSystemProperty(key = IGNITE_CALCITE_USE_QUERY_BLOCKING_TASK_EXECUTOR, value = "true")
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe it's better to test with both executors?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if try to set: value = "false" it pass to dead lock, probably we need to explore it in future, also
if it would be changed: cfg.setQueryThreadPoolSize(11);
from hint:
"The pool size should be greater than the maximum number of concurrent queries initiated by UDFs."
we have 10 parallel threads in test, thus 11 seems to be enough - but don`t or i miss smth ?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, 11 is not enough. Since 10 queries (10 occupied threads) concurrently call customNestedTableFunc (another 10 occupied threads), which call customTableFunc (need to be at least 1 thread to complete each query), so minimum safe amount is 21.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

public class UserDefinedTxAwareFunctionsIntegrationTest extends AbstractBasicIntegrationTest {
/** */
private static final int THREAD_NUM = 10;

/** {@inheritDoc} */
@Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);

cfg.getSqlConfiguration().setQueryEnginesConfiguration(new CalciteQueryEngineConfiguration());
cfg.getTransactionConfiguration().setTxAwareQueriesEnabled(true);
cfg.setQueryThreadPoolSize(2 * THREAD_NUM + 1);

return cfg;
}

/** Check tx aware UDF execution results. */
@Test
public void testTxAwareUserDefinedFunc() {
assertTrue(nodeCount() > 1);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Override nodeCount() instead?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it just a defence if someone try to change it in future (for example - startup speedup with only 1 node)

int nodeCnt = nodeCount();

List<List<Object>> refResults = new ArrayList<>();

IgniteCache<Integer, Object> cache = client.getOrCreateCache(cacheConfig());

refResults.add(List.of(0, Integer.toString(0)));
// Insert outside tx.
cache.query(new SqlFieldsQuery("INSERT INTO Employer(id, name) VALUES (?, ?)").setArgs(0, 0)).getAll();

try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
for (int i = 1; i < 2 * nodeCnt; ++i) {
refResults.add(List.of(i, Integer.toString(i)));
cache.query(new SqlFieldsQuery("INSERT INTO Employer(id, name) VALUES (?, ?)").setArgs(i, i)).getAll();
}

// Simple select without UDF.
List<List<?>> selectResult = cache
.query(new SqlFieldsQuery("SELECT id, name FROM Employer ORDER BY id"))
.getAll();

assertThat(selectResult, equalTo(refResults));

// Select with UDF.
List<List<?>> res = cache.query(new SqlFieldsQuery("SELECT customTableFunc() AS result")).getAll();

assertThat(res.get(0).get(0), equalTo(refResults));

// Select with nested UDF.
res = cache.query(new SqlFieldsQuery("SELECT customNestedTableFunc() AS result")).getAll();

assertThat(res.get(0).get(0), equalTo(refResults));

// UDF participate in DML.
cache.query(new SqlFieldsQuery("INSERT INTO Employer(id, name) VALUES (100, nameAsStr(1))")).getAll();

res = cache.query(new SqlFieldsQuery("SELECT name FROM Employer WHERE id = 100")).getAll();

assertEquals("1", res.get(0).get(0));

for (int i = 0; i < 2 * nodeCnt; ++i) {
// A bit different case of UDF.
List<List<?>> res1 = cache.query(new SqlFieldsQuery("SELECT nameTableFunc(?) AS result").setArgs(i)).getAll();

List<List<?>> res2 = (List<List<?>>)res1.get(0).get(0);

assertThat(res2.get(0).get(0), equalTo(Integer.toString(i)));
}

tx.commit();
}
}

/** */
@Test
public void testIsolationCorrectnessWithUdf() throws IgniteCheckedException {
assertTrue(nodeCount() > 1);
int nodeCnt = nodeCount();

IgniteCache<Integer, Object> cache = client.getOrCreateCache(cacheConfig());

/* The pool size should be greater than the maximum number of concurrent queries initiated by UDFs. */
IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(() -> {
for (int iter = 0; iter < 10; ++iter) {
try (Transaction tx = client.transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
List<List<Object>> refResults = new ArrayList<>();

for (int i = 0; i < 2 * nodeCnt; ++i) {
refResults.add(List.of(i, Integer.toString(i)));
cache.query(new SqlFieldsQuery("INSERT INTO Employer(id, name) VALUES (?, ?)").setArgs(i, i)).getAll();
}

List<List<?>> res = cache.query(new SqlFieldsQuery("SELECT customNestedTableFunc() AS result")).getAll();

assertThat(res.get(0).get(0), equalTo(refResults));

tx.rollback();
}
}
}, THREAD_NUM, "calcite-tx-with-udf");

fut.get(30_000);
}

/** */
private CacheConfiguration<Integer, Object> cacheConfig() {
return this.<Integer, Object>cacheConfiguration()
.setName(DEFAULT_CACHE_NAME)
.setQueryEntities(F.asList(new QueryEntity(Integer.class, Employer.class)
.setTableName("Employer")
.addQueryField("ID", Integer.class.getName(), null)
.setKeyFieldName("ID")
))
.setSqlFunctionClasses(InnerSqlFunctionsLibrary.class)
.setAtomicityMode(TRANSACTIONAL);
}

/** */
public static class InnerSqlFunctionsLibrary {
/** */
@QuerySqlFunction
public List<List<?>> customTableFunc() {
Ignite ignite = Ignition.localIgnite();

return ignite.cache(DEFAULT_CACHE_NAME)
.query(new SqlFieldsQuery("SELECT id, name FROM Employer ORDER BY id"))
.getAll();
}

/** */
@QuerySqlFunction
public List<List<?>> customNestedTableFunc() {
Ignite ignite = Ignition.localIgnite();

Object res = ignite.cache(DEFAULT_CACHE_NAME)
.query(new SqlFieldsQuery("SELECT customTableFunc() AS result"))
.getAll().get(0).get(0);

return (List<List<?>>)res;
}

/** */
@QuerySqlFunction
public static List<List<?>> nameTableFunc(int id) {
Ignite ignite = Ignition.localIgnite();

return ignite.cache(DEFAULT_CACHE_NAME)
.query(new SqlFieldsQuery("SELECT name FROM Employer WHERE id = ?").setArgs(id))
.getAll();
}

/** */
@QuerySqlFunction
public static String nameAsStr(int id) {
Ignite ignite = Ignition.localIgnite();

List<List<?>> res = ignite.cache(DEFAULT_CACHE_NAME)
.query(new SqlFieldsQuery("SELECT name FROM Employer WHERE id = ?").setArgs(id))
.getAll();

return (String)res.get(0).get(0);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -353,7 +353,8 @@ private Node<Object[]> implementFragment(
NoOpIoTracker.INSTANCE,
0,
Commons.parametersMap(ctx.parameters()),
null);
null
);

return new LogicalRelImplementor<>(ectx, c -> r -> 0, mailboxRegistry, exchangeSvc,
new TestFailureProcessor(kernal)).go(fragment.root());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,7 @@
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.UserDefinedTxAwareFunctionsIntegrationTest;
import org.apache.ignite.internal.processors.query.calcite.integration.ViewsIntegrationTest;
import org.apache.ignite.internal.processors.query.calcite.integration.tpch.TpchScale001Test;
import org.apache.ignite.internal.processors.query.calcite.integration.tpch.TpchScale010Test;
Expand Down Expand Up @@ -177,6 +178,7 @@
CacheStoreTest.class,
MultiDcQueryMappingTest.class,
TxWithExceptionalInterceptorTest.class,
UserDefinedTxAwareFunctionsIntegrationTest.class,
CacheWithInterceptorIntegrationTest.class,
TxWithExceptionalInterceptorTest.class,
SelectByKeyFieldTest.class,
Expand Down
Loading