Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IGNITE-24053 Jdbc. Use single observable time tracker for multiple JDBC connections #5069

Open
wants to merge 9 commits into
base: main
Choose a base branch
from
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
/**
* JDBC batch execute request.
*/
public class JdbcBatchExecuteRequest implements ClientMessage {
public class JdbcBatchExecuteRequest extends JdbcObservableTimeAwareRequest implements ClientMessage {
/** Schema name. */
private String schemaName;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
/**
* JDBC prepared statement query batch execute request.
*/
public class JdbcBatchPreparedStmntRequest implements ClientMessage {
public class JdbcBatchPreparedStmntRequest extends JdbcObservableTimeAwareRequest implements ClientMessage {
/** Schema name. */
private String schemaName;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,29 @@

package org.apache.ignite.internal.jdbc.proto.event;

import org.apache.ignite.internal.hlc.HybridTimestamp;
import org.jetbrains.annotations.Nullable;

/**
* Result of commit/rollback command.
*/
public class JdbcFinishTxResult extends Response {
/** Observable timestamp used only on server side. */
@SuppressWarnings("TransientFieldInNonSerializableClass")
private final transient @Nullable HybridTimestamp observableTime;

/**
* Default constructor is used for deserialization.
*/
public JdbcFinishTxResult() {
// No-op.
this.observableTime = null;
}

/**
* Constructor.
*/
public JdbcFinishTxResult(@Nullable HybridTimestamp observableTime) {
this.observableTime = observableTime;
}

/**
Expand All @@ -36,5 +50,14 @@ public JdbcFinishTxResult() {
*/
public JdbcFinishTxResult(int status, String err) {
super(status, err);

this.observableTime = null;
}

/**
* Returns transaction observable time.
*/
public @Nullable HybridTimestamp observableTime() {
return observableTime;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.jdbc.proto.event;

import org.apache.ignite.internal.hlc.HybridTimestampTracker;

/**
* An extension to JDBC request that provides the ability to update and read client observable time.
*/
abstract class JdbcObservableTimeAwareRequest {
/** Tracker of the latest time observed by client. */
@SuppressWarnings("TransientFieldInNonSerializableClass")
private final transient HybridTimestampTracker timestampTracker = HybridTimestampTracker.atomicTracker(null);

/** Returns the tracker of the latest time observed by client. */
public HybridTimestampTracker timestampTracker() {
return timestampTracker;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
/**
* JDBC query execute request.
*/
public class JdbcQueryExecuteRequest implements ClientMessage {
public class JdbcQueryExecuteRequest extends JdbcObservableTimeAwareRequest implements ClientMessage {
/** Expected statement type. */
private JdbcStatementType stmtType;

Expand Down Expand Up @@ -61,6 +61,9 @@ public class JdbcQueryExecuteRequest implements ClientMessage {
*/
private long correlationToken;

/** The latest time observed by client. */
private long observableTime;

/**
* Default constructor. For deserialization purposes.
*/
Expand Down Expand Up @@ -91,7 +94,8 @@ public JdbcQueryExecuteRequest(
boolean autoCommit,
boolean multiStatement,
long queryTimeoutMillis,
long correlationToken
long correlationToken,
long observableTime
) {
Objects.requireNonNull(stmtType);

Expand All @@ -105,6 +109,7 @@ public JdbcQueryExecuteRequest(
this.multiStatement = multiStatement;
this.queryTimeoutMillis = queryTimeoutMillis;
this.correlationToken = correlationToken;
this.observableTime = observableTime;
}

/**
Expand Down Expand Up @@ -190,6 +195,10 @@ public long correlationToken() {
return correlationToken;
}

public long observableTime() {
return observableTime;
}

/** {@inheritDoc} */
@Override
public void writeBinary(ClientMessagePacker packer) {
Expand All @@ -204,6 +213,7 @@ public void writeBinary(ClientMessagePacker packer) {
packer.packObjectArrayAsBinaryTuple(args);
packer.packLong(queryTimeoutMillis);
packer.packLong(correlationToken);
packer.packLong(observableTime);
}

/** {@inheritDoc} */
Expand All @@ -220,6 +230,7 @@ public void readBinary(ClientMessageUnpacker unpacker) {
args = unpacker.unpackObjectArrayFromBinaryTuple();
queryTimeoutMillis = unpacker.unpackLong();
correlationToken = unpacker.unpackLong();
observableTime = unpacker.unpackLong();
}

/** {@inheritDoc} */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,9 +39,6 @@ public abstract class Response implements ClientMessage {
/** Error. */
private String err;

/** Has results. */
protected boolean hasResults;

/**
* Constructs successful response.
*/
Expand Down Expand Up @@ -121,9 +118,9 @@ public void err(String err) {
}

/**
* Gets hasResults flag.
* Gets success status.
*
* @return Has results.
* @return {@code True} if command succeeded, {@code false} otherwise.
*/
public boolean success() {
return status == STATUS_SUCCESS;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -715,10 +715,6 @@ private void processOperation(ChannelHandlerContext ctx, ClientMessageUnpacker i
return ClientTupleContainsAllKeysRequest.process(in, out, igniteTables, resources, txManager);

case ClientOp.JDBC_CONNECT:
// TODO: IGNITE-24053 JDBC request ought to contain the client observation timestamp.
jdbcQueryEventHandler.getTimestampTracker().update(clockService.current());
out.meta(jdbcQueryEventHandler.getTimestampTracker().get());

return ClientJdbcConnectRequest.execute(in, out, jdbcQueryEventHandler);

case ClientOp.JDBC_EXEC:
Expand Down Expand Up @@ -821,6 +817,7 @@ private void processOperation(ChannelHandlerContext ctx, ClientMessageUnpacker i
case ClientOp.SQL_EXEC_SCRIPT:
return ClientSqlExecuteScriptRequest.process(in, queryProcessor).thenRun(() -> {
if (out.meta() == null) {
// TODO https://issues.apache.org/jira/browse/IGNITE-24275 Must set updated time instead of current time.
out.meta(clockService.current());
}
});
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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.client.handler;

import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;

import java.time.ZoneId;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.ignite.internal.hlc.HybridTimestamp;
import org.apache.ignite.internal.hlc.HybridTimestampTracker;
import org.apache.ignite.internal.tx.InternalTransaction;
import org.apache.ignite.internal.tx.InternalTxOptions;
import org.apache.ignite.internal.tx.TxManager;
import org.apache.ignite.lang.CancelHandle;
import org.apache.ignite.lang.CancellationToken;
import org.jetbrains.annotations.Nullable;

/**
* JDBC connection context.
*/
class JdbcConnectionContext {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

this class is simply moved to the top level

private final AtomicBoolean closed = new AtomicBoolean();

private final Object mux = new Object();

private final TxManager txManager;

private final ZoneId timeZoneId;

private final ConcurrentMap<Long, CancelHandle> cancelHandles = new ConcurrentHashMap<>();

private @Nullable InternalTransaction tx;

JdbcConnectionContext(
TxManager txManager,
ZoneId timeZoneId
) {
this.txManager = txManager;
this.timeZoneId = timeZoneId;
}

ZoneId timeZoneId() {
return timeZoneId;
}

/**
* Gets the transaction associated with the current connection, starts a new one if it doesn't already exist.
*
* <p>NOTE: this method is not thread-safe and should only be called by a single thread.
*
* @return Transaction associated with the current connection.
*/
InternalTransaction getOrStartTransaction(HybridTimestampTracker timestampTracker) {
if (tx == null) {
tx = txManager.beginExplicitRw(timestampTracker, InternalTxOptions.defaults());
}

return tx;
}

/**
* Finishes active transaction, if one exists.
*
* <p>NOTE: this method is not thread-safe and should only be called by a single thread.
*
* @param commit {@code True} to commit, {@code false} to rollback.
* @return Future that represents the pending completion of the operation.
*/
CompletableFuture<@Nullable HybridTimestamp> finishTransactionAsync(boolean commit) {
InternalTransaction tx0 = tx;

tx = null;

if (tx0 == null) {
return nullCompletedFuture();
}

return commit
? tx0.commitAsync().thenApply(ignore -> tx0.observableTimestamp())
: tx0.rollbackAsync().thenApply(ignore -> null);
}

boolean valid() {
return !closed.get();
}

void close() {
if (!closed.compareAndSet(false, true)) {
return;
}

synchronized (mux) {
finishTransactionAsync(false);
}
}

CancellationToken registerExecution(long token) {
CancelHandle handle = CancelHandle.create();

CancelHandle previousHandle = cancelHandles.putIfAbsent(token, handle);

assert previousHandle == null;

return handle.token();
}

void deregisterExecution(long token) {
cancelHandles.remove(token);
}

CompletableFuture<Void> cancelExecution(long token) {
CancelHandle handle = cancelHandles.remove(token);

if (handle == null) {
return nullCompletedFuture();
}

return handle.cancelAsync();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

import static org.apache.ignite.internal.jdbc.proto.IgniteQueryErrorCode.UNSUPPORTED_OPERATION;
import static org.apache.ignite.internal.sql.engine.SqlQueryType.DDL;
import static org.apache.ignite.internal.sql.engine.SqlQueryType.DML;
import static org.apache.ignite.internal.sql.engine.SqlQueryType.KILL;
import static org.apache.ignite.internal.sql.engine.SqlQueryType.TX_CONTROL;

Expand Down Expand Up @@ -49,16 +48,12 @@
* Contains common methods used to process jdbc requests.
*/
abstract class JdbcHandlerBase {

/** {@link SqlQueryType}s allowed in JDBC select statements. **/
public static final Set<SqlQueryType> SELECT_STATEMENT_QUERIES = Set.of(
SqlQueryType.QUERY,
SqlQueryType.EXPLAIN
);

/** {@link SqlQueryType}s allowed in JDBC update statements. **/
public static final Set<SqlQueryType> UPDATE_STATEMENT_QUERIES = Set.of(DML, DDL, KILL);
Copy link
Contributor Author

Choose a reason for hiding this comment

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

was not used


/** {@link SqlQueryType}s types that return 0 in executeUpdate and execute / getUpdateCount. **/
public static final Set<SqlQueryType> ZERO_UPDATE_COUNT_QUERIES = Set.of(DDL, KILL, TX_CONTROL);

Expand Down
Loading