You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/11/19 05:58:53 UTC

[GitHub] [ignite-3] agura commented on a change in pull request #326: IGNITE-15212 Add SQL API

agura commented on a change in pull request #326:
URL: https://github.com/apache/ignite-3/pull/326#discussion_r752475233



##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Session.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.sql.async.AsyncSession;
+import org.apache.ignite.sql.reactive.ReactiveSession;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * SQL Session provides methods for query execution.
+ */
+public interface Session extends AsyncSession, ReactiveSession {
+    /**
+     * Sets default query timeout.
+     *
+     * @param timeout  Query timeout value.
+     * @param timeUnit Timeunit.
+     */
+    void defaultTimeout(int timeout, TimeUnit timeUnit);
+    
+    /**
+     * Gets default query timeout.
+     *
+     * @param timeUnit Timeunit.
+     * @return Default query timeout.
+     */
+    long defaultTimeout(TimeUnit timeUnit);
+    
+    /**
+     * Sets default query schema.
+     *
+     * @param schema Default schema.
+     */
+    void defaultSchema(@NotNull String schema);
+    
+    /**
+     * Gets default query schema.
+     *
+     * @return Default query schema.
+     */
+    String defaultSchema();
+    
+    /**
+     * Executes single SQL query.
+     *
+     * @param query     SQL query template.
+     * @param arguments Arguments for the template (optional).
+     * @return SQL query results set.
+     * @throws SqlException If failed.
+     */
+    ResultSet execute(@NotNull String query, Object... arguments);
+    
+    /**
+     * Executes single SQL statement.
+     *
+     * @param statement SQL statement to execute.
+     * @return SQL query results set.
+     */
+    ResultSet execute(@NotNull Statement statement);
+    
+    /**
+     * Executes multi-statement SQL query.
+     *
+     * @param query     SQL query template.
+     * @param arguments Arguments for the template (optional).
+     * @return SQL query results set.
+     * @throws SqlException If failed.
+     */
+    MultiResultSet executeScript(@NotNull String query, Object... arguments);
+    
+    /**
+     * Sets session property.
+     *
+     * @param name  Property name.
+     * @param value Property value.
+     * @return {@code this} for chaining.
+     */
+    Session property(@NotNull String name, Object value);

Review comment:
       What is a session property? Could I change the property while use some session instance? Why I can't read the session property?

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Session.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.sql.async.AsyncSession;
+import org.apache.ignite.sql.reactive.ReactiveSession;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * SQL Session provides methods for query execution.
+ */
+public interface Session extends AsyncSession, ReactiveSession {
+    /**
+     * Sets default query timeout.
+     *
+     * @param timeout  Query timeout value.
+     * @param timeUnit Timeunit.
+     */
+    void defaultTimeout(int timeout, TimeUnit timeUnit);
+    
+    /**
+     * Gets default query timeout.
+     *
+     * @param timeUnit Timeunit.
+     * @return Default query timeout.
+     */
+    long defaultTimeout(TimeUnit timeUnit);
+    
+    /**
+     * Sets default query schema.
+     *
+     * @param schema Default schema.
+     */
+    void defaultSchema(@NotNull String schema);
+    
+    /**
+     * Gets default query schema.
+     *
+     * @return Default query schema.
+     */
+    String defaultSchema();
+    
+    /**
+     * Executes single SQL query.
+     *
+     * @param query     SQL query template.
+     * @param arguments Arguments for the template (optional).
+     * @return SQL query results set.
+     * @throws SqlException If failed.
+     */
+    ResultSet execute(@NotNull String query, Object... arguments);
+    
+    /**
+     * Executes single SQL statement.
+     *
+     * @param statement SQL statement to execute.
+     * @return SQL query results set.
+     */
+    ResultSet execute(@NotNull Statement statement);

Review comment:
       Would it be useful to have short-cut method `ResultSet execute(@NotNull Statement statement, Object... arguments)`?

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Statement.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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * The object represents parameterized SQL query statement that supports batched query, and provides methods for managing it`s state.
+ */
+public interface Statement {
+    /**
+     * Returns SQL statement string representation.
+     *
+     * @return SQL statement string.
+     */
+    String query();
+    
+    /**
+     * Returns SQL statement parameters.
+     *
+     * @return SQL statement parameters.
+     */
+    Object[] parameters();
+    
+    /**
+     * Sets SQL statement parameters.
+     *
+     * @param parameters SQL statement parameters.
+     * @return {@code this} for chaining.
+     */
+    Statement parameters(Object... parameters);
+    
+    /**
+     * Sets a SQL statement parameter value by the parameter index.
+     *
+     * @param parameterIndex Parameter index.
+     * @param parameterValue Parameter value.
+     * @return {@code this} for chaining.
+     */
+    Statement parameter(int parameterIndex, Object parameterValue);

Review comment:
       Just a cosmetic. `parameter` word is definitely redundant in method argument names. 

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/SqlRow.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.sql;
+
+import org.apache.ignite.table.Tuple;
+
+/**
+ * SQL row provides methods to access row data by column name or id.
+ *
+ * <p>Column description can be retrived from {@link ResultSet#metadata()}.

Review comment:
       retri**e**ved

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/ResultSet.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.sql;
+
+/**
+ * SQL result set provides methods to access SQL query result represented as collection of {@link
+ * SqlRow}.
+ *
+ * <p>All the rows in result set have the same structure described in {@link ResultSetMetadata}.
+ * ResultSet must be closed after usage to free resources.
+ */
+public interface ResultSet extends Iterable<SqlRow>, AutoCloseable {
+    /**
+     * Returns metadata for the results.
+     *
+     * @return ResultSet metadata.
+     */
+    ResultSetMetadata metadata();
+    
+    /**
+     * Returns whether the result set contains rows (SELECT query result), or not (for query of DML, DDL or other kind).
+     *
+     * @return {@code True} if result set contains rows, {@code false} otherwise.
+     */
+    boolean hasRowSet();
+    
+    /**
+     * Returns number of row affected by DML query.
+     *
+     * @return Number of rows or {@code -1} if unapplicable.
+     */
+    int updateCount();
+    
+    /**
+     * Returns result for the conditional query.
+     *
+     * @return {@code True} if conditional query applied, {@code false} otherwise.
+     */
+    boolean wasApplied();

Review comment:
       It seems that statement `wasApplied == true` equals to `updateCount == 0`. Isn't it?

##########
File path: modules/api/src/main/java/org/apache/ignite/table/IgniteTableStatistics.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.table;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Table statistics facade provides methods for managing table statistics.
+ *
+ * <p>Table statistics are used by SQL engine for SQL queries planning.
+ */
+// TODO: drop this class and move all the methods to Table interface?
+public interface IgniteTableStatistics {

Review comment:
       Why is this interface not internal? 

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/MultiResultSet.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.sql;
+
+/**
+ * Result set for multi-statement query.
+ */
+public interface MultiResultSet extends Iterable<ResultSet>, AutoCloseable {

Review comment:
       Need usage example. I don't have any idea how it should be used.

##########
File path: modules/api/src/main/java/org/apache/ignite/table/IgniteTableStatistics.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.table;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Table statistics facade provides methods for managing table statistics.
+ *
+ * <p>Table statistics are used by SQL engine for SQL queries planning.
+ */
+// TODO: drop this class and move all the methods to Table interface?
+public interface IgniteTableStatistics {
+    /**
+     * Returns all the statistics names for the table.
+     *
+     * @return Statistics names for the table.
+     */
+    Collection<String> statisticNames();
+    
+    /**
+     * Get statistics info for the table.
+     *
+     * @param statisticName Statistic name.
+     * @return Statistics info.
+     */
+    StatisticInfo statistic(String statisticName); //TODO: Local or global? Ready or in-progress? TBD.

Review comment:
       There is nothing about "local or global" and "ready or in-progress" terms in javadoc. What do these terms mean?

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Session.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.sql.async.AsyncSession;
+import org.apache.ignite.sql.reactive.ReactiveSession;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * SQL Session provides methods for query execution.

Review comment:
       Is it possible that `Session` instance is shred between several threads? It should be documented.
   
   Answer to this question also could lead to new questions about timeouts, properties and transactions.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/ResultSet.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.sql;
+
+/**
+ * SQL result set provides methods to access SQL query result represented as collection of {@link
+ * SqlRow}.
+ *
+ * <p>All the rows in result set have the same structure described in {@link ResultSetMetadata}.
+ * ResultSet must be closed after usage to free resources.
+ */
+public interface ResultSet extends Iterable<SqlRow>, AutoCloseable {
+    /**
+     * Returns metadata for the results.
+     *
+     * @return ResultSet metadata.
+     */
+    ResultSetMetadata metadata();
+    
+    /**
+     * Returns whether the result set contains rows (SELECT query result), or not (for query of DML, DDL or other kind).

Review comment:
       There is a question here: could DML statement return some value or tuple? If yes, then javadoc should be rewritten.
   
   Also, is there a difference between empty result set (e.g. no rows satisfying to a query returned) and result set without rows (hasRowSet == true)? What is result of `hasRowSet()` invocation for case where some select didn't return any row for given condition?

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/ResultSet.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.sql;
+
+/**
+ * SQL result set provides methods to access SQL query result represented as collection of {@link
+ * SqlRow}.
+ *
+ * <p>All the rows in result set have the same structure described in {@link ResultSetMetadata}.
+ * ResultSet must be closed after usage to free resources.
+ */
+public interface ResultSet extends Iterable<SqlRow>, AutoCloseable {
+    /**
+     * Returns metadata for the results.
+     *
+     * @return ResultSet metadata.
+     */
+    ResultSetMetadata metadata();
+    
+    /**
+     * Returns whether the result set contains rows (SELECT query result), or not (for query of DML, DDL or other kind).
+     *
+     * @return {@code True} if result set contains rows, {@code false} otherwise.
+     */
+    boolean hasRowSet();
+    
+    /**
+     * Returns number of row affected by DML query.

Review comment:
       I suggest to avoid usage of `DML` term in docs if possible. 

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Statement.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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * The object represents parameterized SQL query statement that supports batched query, and provides methods for managing it`s state.
+ */
+public interface Statement {
+    /**
+     * Returns SQL statement string representation.
+     *
+     * @return SQL statement string.
+     */
+    String query();
+    
+    /**
+     * Returns SQL statement parameters.
+     *
+     * @return SQL statement parameters.
+     */
+    Object[] parameters();
+    
+    /**
+     * Sets SQL statement parameters.
+     *
+     * @param parameters SQL statement parameters.
+     * @return {@code this} for chaining.
+     */
+    Statement parameters(Object... parameters);
+    
+    /**
+     * Sets a SQL statement parameter value by the parameter index.
+     *
+     * @param parameterIndex Parameter index.
+     * @param parameterValue Parameter value.
+     * @return {@code this} for chaining.
+     */
+    Statement parameter(int parameterIndex, Object parameterValue);
+    
+    /**
+     * Resets batch state and clears query parameters.
+     *
+     * @return {@code this} for chaining.
+     */
+    Statement resetState();
+    
+    /**
+     * Adds a set of parameters to this statement object's batch of commands.
+     *
+     * @return {@code this} for chaining.
+     */
+    Statement addBatch() throws SqlException;
+    
+    /**
+     * Sets query timeout.
+     *
+     * @param timeout  Query timeout value.
+     * @param timeUnit Timeunit.
+     */
+    void queryTimeout(int timeout, TimeUnit timeUnit);

Review comment:
       Inconsistent API: `timeout` parameter's type is `int`, while `queryTimeout(TimeUnit timeUnit)` returns `long`.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/ColumnMetadata.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.sql;
+
+import org.apache.ignite.schema.definition.ColumnType;
+
+/**
+ * Column metadata.
+ */
+public interface ColumnMetadata {
+    /**
+     * Return column name in the result set.
+     *
+     * <p>Note: If row column does not represent any table column, then generated name will be
+     * used.
+     *
+     * @return Column name.
+     */
+    String name();
+    
+    /**
+     * Returns column type.
+     *
+     * @return Column type.
+     */
+    ColumnType type();
+    
+    /**
+     * Returns column value type.
+     *
+     * @return Value type.
+     */
+    Class<?> valueClass();

Review comment:
       It should be well documented what is the difference between "column type" (see `type()` method javadoc) and "column value type".

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/ResultSetMetadata.java
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.sql;
+
+import java.util.List;
+
+/**
+ * ResultSet metadata.
+ */
+public interface ResultSetMetadata {
+    /**
+     * Returns number of column that every row in a result set.

Review comment:
       It seems "contains" word is missed here.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/Session.java
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.sql;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.sql.async.AsyncSession;
+import org.apache.ignite.sql.reactive.ReactiveSession;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * SQL Session provides methods for query execution.
+ */
+public interface Session extends AsyncSession, ReactiveSession {
+    /**
+     * Sets default query timeout.
+     *
+     * @param timeout  Query timeout value.
+     * @param timeUnit Timeunit.
+     */
+    void defaultTimeout(int timeout, TimeUnit timeUnit);

Review comment:
       Why `timeout` has type `int` while `defaultTimeout(TimeUnit timeUnit)` method returns value of type `long`?

##########
File path: modules/api/src/main/java/org/apache/ignite/table/Table.java
##########
@@ -95,4 +95,15 @@
     default <K, V> KeyValueView<K, V> keyValueView(Class<K> keyCls, Class<V> valCls) {
         return keyValueView(Mapper.of(keyCls), Mapper.of(valCls));
     }
+    
+    /**
+     * Returns statistics facade for table statistics management.
+     *
+     * <p>Note: SQL engine uses table statistics for SQL queries planning.
+     *
+     * @return Statistics facade.
+     */
+    default IgniteTableStatistics statistics() {

Review comment:
       Not sure that this operation should be available on public API.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/async/AsyncResultSet.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.sql.async;
+
+import java.util.concurrent.CompletionStage;
+import org.apache.ignite.sql.ResultSetMetadata;
+import org.apache.ignite.sql.SqlRow;
+
+/**
+ * Asynchronous result set.
+ */
+public interface AsyncResultSet {
+    /**
+     * Returns metadata for the results.
+     *
+     * @return ResultSet metadata.
+     */
+    ResultSetMetadata metadata();
+    
+    /**
+     * Returns whether the result set contains rows (SELECT query result), or not (for query of DML, DDL or other kind).
+     *
+     * @return {@code True} if result set contains rows, {@code false} otherwise.
+     */
+    boolean hasRowSet();
+    
+    /**
+     * Returns number of row affected by DML query.
+     *
+     * @return Number of rows.
+     */
+    int updateCount();
+    
+    /**
+     * Returns result for the conditional query.
+     *
+     * @return {@code True} if conditional query applied, {@code false} otherwise.
+     */
+    boolean wasApplied();
+    
+    /**
+     * Returns the current page content.
+     *
+     * @return Iterable over rows.
+     */
+    Iterable<SqlRow> currentPage();

Review comment:
       What is a page? How could it be configured (e.g. page size)?

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/reactive/ReactiveSession.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.sql.reactive;
+
+import org.apache.ignite.sql.SqlException;
+import org.apache.ignite.sql.Statement;
+
+/**
+ * Reactive Session provides methods for reactive query execution.
+ */
+public interface ReactiveSession {
+    /**
+     * Executes SQL query in reactive way.
+     *
+     * @param sql  SQL query template.
+     * @param args Arguments for the template (optional).
+     * @return Reactive result.
+     * @throws SqlException If failed.
+     */
+    ReactiveResultSet executeReactive(String sql, Object... args);

Review comment:
       Session is already reactive. Suffix `Reactive` is redundant here and below.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/async/AsyncSession.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.sql.async;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.sql.SqlException;
+import org.apache.ignite.sql.Statement;
+
+/**
+ * Async Session provides methods for asynchronous query execution.
+ */
+public interface AsyncSession {
+    /**
+     * Executes SQL query in async way.
+     *
+     * @param query     SQL query template.
+     * @param arguments Arguments for the template (optional).
+     * @return Operation future.
+     * @throws SqlException If failed.
+     */
+    CompletableFuture<AsyncResultSet> executeAsync(String query, Object... arguments);

Review comment:
       If session is async the execute obviously should be async. IMHO `Async` suffix is redundant here and below.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/SqlException.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.sql;
+
+import org.apache.ignite.lang.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * SQL exception base class.
+ */
+//TODO: Do we want to use this instead of java.sql.SQLException ?

Review comment:
       @ascherbakoff Not sure. In case of usage of `java.sql.SQLException` we are restricted JDBC implementation. Moreover, `java.sql.SQLException` is checked, but we have an agreement about usage of unchecked exceptions on public APIs. Also, `java.sql.SQLException` doesn't fit into our conception where we should use specific exception's constructors with codes.

##########
File path: modules/api/src/main/java/org/apache/ignite/sql/async/AsyncSession.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.sql.async;
+
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.sql.SqlException;
+import org.apache.ignite.sql.Statement;
+
+/**
+ * Async Session provides methods for asynchronous query execution.
+ */
+public interface AsyncSession {

Review comment:
       Because we have an agreement about implementing own native SQL API.

##########
File path: modules/sql/src/test/java/IgniteSqlTest.java
##########
@@ -0,0 +1,384 @@
+/*
+ * 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.
+ */
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Flow;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.util.Constants;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.sql.IgniteSql;
+import org.apache.ignite.sql.MultiResultSet;
+import org.apache.ignite.sql.ResultSet;
+import org.apache.ignite.sql.ResultSetMetadata;
+import org.apache.ignite.sql.Session;
+import org.apache.ignite.sql.SqlRow;
+import org.apache.ignite.sql.async.AsyncResultSet;
+import org.apache.ignite.sql.reactive.ReactiveResultSet;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Tuple;
+import org.apache.ignite.tx.IgniteTransactions;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+/**
+ * Tests IgniteSQL facade API.
+ */
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.LENIENT)
+public class IgniteSqlTest {
+    @Mock
+    IgniteSql queryMgr;
+    
+    @Mock
+    private IgniteTransactions igniteTx;
+    
+    @Mock
+    private Transaction tx;
+    
+    @BeforeEach
+    void setUp() {
+        initMock();
+    }
+    
+    @Test
+    public void testSyncSql() {
+        igniteTx.runInTransaction(tx -> {
+            Session sess = queryMgr.newSession();
+            
+            sess.defaultTimeout(10_000, TimeUnit.MILLISECONDS); // Set default timeout.
+            sess.property("memoryQuota", 10 * Constants.MiB); // Set default quota.
+            
+            // Execute outside TX.
+            ResultSet rs = sess.execute("INSERT INTO tbl VALUES (?, ?)", 10, "str");
+            
+            assertEquals(1, rs.updateCount());
+            
+            // Execute in TX.
+            Session txSession = tx.wrap(queryMgr.newSession());
+            
+            rs = txSession.execute("SELECT id, val FROM tbl WHERE id < {};", 10);
+            
+            for (SqlRow r : rs) {
+                assertTrue(10 > r.longValue("id"));
+                assertTrue((r.stringValue("val")).startsWith("str"));
+            }
+            
+            tx.commit();
+        });
+        
+        Mockito.verify(tx).commit();
+    }
+    
+    @Test
+    public void testSyncSql2() {
+        RecordView<Tuple> tbl = getTable();
+        
+        // Starts new TX.
+        Session txSession = tx.wrap(queryMgr.newSession());
+        
+        ResultSet rs = txSession.execute("SELECT id, val FROM tbl WHERE id < {};", 10);
+        SqlRow row = rs.iterator().next();
+        
+        tbl.withTransaction(tx)
+                .insertAsync(Tuple.create().set("val", "NewValue"))
+                .thenAccept(r -> txSession.transaction().rollback());
+        
+        Mockito.verify(tx, Mockito.times(1)).rollback();
+    }
+    
+    @Test
+    public void testSyncMultiStatementSql() {
+        Session sess = tx.wrap(queryMgr.newSession());
+        
+        MultiResultSet multiRs = sess.executeScript(
+                "CREATE TABLE tbl(id INTEGER PRIMARY KEY, val VARCHAR);"
+                        + "INSERT INTO tbl VALUES (1, 2);"
+                        + "SELECT id, val FROM tbl WHERE id == {};"
+                        + "DROP TABLE tbl", 10);
+        
+        Iterator<ResultSet> iterator = multiRs.iterator();
+        //TODO: Can iterator return null??

Review comment:
       As I remember we have an agreement about an impossibility of usage DDL and SQL statements in the same transaction. Need to discuss again.

##########
File path: modules/sql/src/test/java/IgniteSqlTest.java
##########
@@ -0,0 +1,384 @@
+/*
+ * 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.
+ */
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Flow;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.schema.SchemaRegistry;
+import org.apache.ignite.internal.util.Constants;
+import org.apache.ignite.schema.definition.ColumnType;
+import org.apache.ignite.sql.IgniteSql;
+import org.apache.ignite.sql.MultiResultSet;
+import org.apache.ignite.sql.ResultSet;
+import org.apache.ignite.sql.ResultSetMetadata;
+import org.apache.ignite.sql.Session;
+import org.apache.ignite.sql.SqlRow;
+import org.apache.ignite.sql.async.AsyncResultSet;
+import org.apache.ignite.sql.reactive.ReactiveResultSet;
+import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Tuple;
+import org.apache.ignite.tx.IgniteTransactions;
+import org.apache.ignite.tx.Transaction;
+import org.jetbrains.annotations.NotNull;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import org.mockito.junit.jupiter.MockitoExtension;
+import org.mockito.junit.jupiter.MockitoSettings;
+import org.mockito.quality.Strictness;
+
+/**
+ * Tests IgniteSQL facade API.
+ */
+@ExtendWith(MockitoExtension.class)
+@MockitoSettings(strictness = Strictness.LENIENT)
+public class IgniteSqlTest {
+    @Mock
+    IgniteSql queryMgr;
+    
+    @Mock
+    private IgniteTransactions igniteTx;
+    
+    @Mock
+    private Transaction tx;
+    
+    @BeforeEach
+    void setUp() {
+        initMock();
+    }
+    
+    @Test
+    public void testSyncSql() {
+        igniteTx.runInTransaction(tx -> {
+            Session sess = queryMgr.newSession();
+            
+            sess.defaultTimeout(10_000, TimeUnit.MILLISECONDS); // Set default timeout.
+            sess.property("memoryQuota", 10 * Constants.MiB); // Set default quota.
+            
+            // Execute outside TX.
+            ResultSet rs = sess.execute("INSERT INTO tbl VALUES (?, ?)", 10, "str");
+            
+            assertEquals(1, rs.updateCount());
+            
+            // Execute in TX.
+            Session txSession = tx.wrap(queryMgr.newSession());
+            
+            rs = txSession.execute("SELECT id, val FROM tbl WHERE id < {};", 10);
+            
+            for (SqlRow r : rs) {
+                assertTrue(10 > r.longValue("id"));
+                assertTrue((r.stringValue("val")).startsWith("str"));
+            }
+            
+            tx.commit();
+        });
+        
+        Mockito.verify(tx).commit();
+    }
+    
+    @Test
+    public void testSyncSql2() {
+        RecordView<Tuple> tbl = getTable();
+        
+        // Starts new TX.
+        Session txSession = tx.wrap(queryMgr.newSession());
+        
+        ResultSet rs = txSession.execute("SELECT id, val FROM tbl WHERE id < {};", 10);
+        SqlRow row = rs.iterator().next();
+        
+        tbl.withTransaction(tx)
+                .insertAsync(Tuple.create().set("val", "NewValue"))
+                .thenAccept(r -> txSession.transaction().rollback());
+        
+        Mockito.verify(tx, Mockito.times(1)).rollback();
+    }
+    
+    @Test
+    public void testSyncMultiStatementSql() {
+        Session sess = tx.wrap(queryMgr.newSession());
+        
+        MultiResultSet multiRs = sess.executeScript(
+                "CREATE TABLE tbl(id INTEGER PRIMARY KEY, val VARCHAR);"
+                        + "INSERT INTO tbl VALUES (1, 2);"
+                        + "SELECT id, val FROM tbl WHERE id == {};"
+                        + "DROP TABLE tbl", 10);
+        
+        Iterator<ResultSet> iterator = multiRs.iterator();
+        //TODO: Can iterator return null??

Review comment:
       Whan does `null` mean for `ResultSet`?

##########
File path: modules/api/src/main/java/org/apache/ignite/table/IgniteTableStatistics.java
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.table;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Table statistics facade provides methods for managing table statistics.
+ *
+ * <p>Table statistics are used by SQL engine for SQL queries planning.
+ */
+// TODO: drop this class and move all the methods to Table interface?
+public interface IgniteTableStatistics {
+    /**
+     * Returns all the statistics names for the table.
+     *
+     * @return Statistics names for the table.
+     */
+    Collection<String> statisticNames();
+    
+    /**
+     * Get statistics info for the table.
+     *
+     * @param statisticName Statistic name.
+     * @return Statistics info.
+     */
+    StatisticInfo statistic(String statisticName); //TODO: Local or global? Ready or in-progress? TBD.
+    
+    /**
+     * Creates statistics for the table and initiate it gathering on the nodes.
+     *
+     * @param statisticsConfiguration Statistic configuration.
+     * @return Operation future.
+     */
+    CompletableFuture<StatisticInfo> gather(StatisticConfiguration statisticsConfiguration);

Review comment:
       May be `collect` or `refresh` instead of `gather`?

##########
File path: modules/api/src/main/java/org/apache/ignite/tx/Transaction.java
##########
@@ -46,4 +47,20 @@
      * @return The future.
      */
     CompletableFuture<Void> rollbackAsync();
+    
+    /**
+     * Enlists SQL session into a transaction.
+     *
+     * @param session SQL session.
+     * @return Enlisted session.
+     */
+    Session wrap(Session session);

Review comment:
       "Enlists SQL session into a transaction."
   
   So, may be method should have name `enlist`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org