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-24274: Sql. Inbox does not handle closing properly #5103

Open
wants to merge 6 commits into
base: main
Choose a base branch
from
Open
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 @@ -42,6 +42,7 @@
import org.apache.ignite.internal.sql.engine.exec.exp.ExpressionFactory;
import org.apache.ignite.internal.sql.engine.exec.mapping.ColocationGroup;
import org.apache.ignite.internal.sql.engine.exec.mapping.FragmentDescription;
import org.apache.ignite.internal.sql.engine.exec.rel.Node;
import org.apache.ignite.internal.sql.engine.prepare.pruning.PartitionPruningColumns;
import org.apache.ignite.internal.sql.engine.prepare.pruning.PartitionPruningMetadata;
import org.apache.ignite.internal.sql.engine.schema.IgniteTable;
Expand Down Expand Up @@ -324,7 +325,7 @@ public SharedState sharedState() {
}

/**
* Executes a query task.
* Executes a query task. To execute a task from a {@link Node} use {@link Node#execute(RunnableX)} instead.
*
* @param task Query task.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;

import java.util.List;
import org.apache.ignite.internal.lang.RunnableX;
import org.apache.ignite.internal.sql.engine.QueryCancelledException;
import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
import org.apache.ignite.internal.sql.engine.util.Commons;
Expand Down Expand Up @@ -105,6 +106,22 @@ public void rewind() {
}
}

/** {@inheritDoc} */
@Override
public void execute(RunnableX task) {
if (this.isClosed()) {
return;
}

context().execute(() -> {
// If the node is closed, the task must be ignored.
if (this.isClosed()) {
return;
}
task.run();
}, this::onError);
}

/** {@inheritDoc} */
@Override
public void onRegister(Downstream<RowT> downstream) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::doJoin, this::onError);
this.execute(this::doJoin);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public void request(int rowsCnt) throws Exception {
if (waiting == 0) {
sources().get(curSrcIdx).request(waiting = inBufSize);
} else if (!inLoop) {
context().execute(this::flush, this::onError);
this.execute(this::flush);
}
}

Expand Down Expand Up @@ -179,7 +179,7 @@ private void flush() throws Exception {

if (processed >= inBufSize && requested > 0) {
// Allow others to do their job.
context().execute(this::flush, this::onError);
this.execute(this::flush);

return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -268,11 +268,11 @@ private void flush() throws Exception {
*/
private void scheduleTask() {
if (!pendingRequests.isEmpty() && taskScheduled.compareAndSet(false, true)) {
source.context().execute(() -> {
source.execute(() -> {
taskScheduled.set(false);

flush();
}, source::onError);
});
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -258,12 +258,12 @@ private void onRequest() throws Exception {
assert nullOrEmpty(leftInBuf);
assert nullOrEmpty(rightInBuf);

context().execute(() -> {
this.execute(() -> {
checkState();

state = State.FILLING_LEFT;
leftSource().request(waitingLeft = leftInBufferSize);
}, this::onError);
});

break;
case IDLE:
Expand All @@ -272,11 +272,11 @@ private void onRequest() throws Exception {
assert waitingRight == -1 || waitingRight == 0 && rightInBuf.size() == rightInBufferSize;
assert waitingLeft == -1 || waitingLeft == 0 && leftInBuf.size() == leftInBufferSize;

context().execute(() -> {
this.execute(() -> {
checkState();

join();
}, this::onError);
});

break;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::doFilter, this::onError);
this.execute(this::doFilter);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,7 +107,7 @@ public void request(int rowsCnt) throws Exception {
if (waiting == 0) {
source().request(waiting = inBufSize);
} else if (!inLoop) {
context().execute(this::flush, this::onError);
this.execute(this::flush);
}
}

Expand Down Expand Up @@ -191,7 +191,7 @@ private void flush() throws Exception {

if (processed >= inBufSize && requested > 0) {
// allow others to do their job
context().execute(this::flush, this::onError);
this.execute(this::flush);

return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::doPush, this::onError);
this.execute(this::doPush);
}
}

Expand Down Expand Up @@ -356,7 +356,7 @@ private void requestBatches(String nodeName, int cnt, @Nullable SharedState stat
ex
);

context().execute(() -> onError(wrapperEx), this::onError);
this.execute(() -> onError(wrapperEx));
}
});
}
Expand All @@ -367,9 +367,9 @@ private void requestBatches(String nodeName, int cnt, @Nullable SharedState stat
*/
public void onNodeLeft(String nodeName) {
if (context().originatingNodeName().equals(nodeName) && srcNodeNames == null) {
context().execute(this::close, this::onError);
this.execute(this::close);
} else if (srcNodeNames != null && srcNodeNames.contains(nodeName)) {
context().execute(() -> onNodeLeft0(nodeName), this::onError);
this.execute(() -> onNodeLeft0(nodeName));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ public void push(RowT row) throws Exception {
waiting--;

if (waiting == 0) {
context().execute(this::requestSource, this::onError);
this.execute(this::requestSource);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::doJoin, this::onError);
this.execute(this::doJoin);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,7 +290,7 @@ private void flushTuples() {
throw new UnsupportedOperationException(modifyOp.name());
}

modifyResult.whenComplete((r, e) -> context().execute(() -> {
modifyResult.whenComplete((r, e) -> this.execute(() -> {
if (e != null) {
onError(e);

Expand All @@ -308,7 +308,7 @@ private void flushTuples() {
requestNextBatchIfNeeded();

tryEnd();
}, this::onError));
}));
}

private boolean needToFlush() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.ignite.internal.sql.engine.exec.rel;

import java.util.List;
import org.apache.ignite.internal.lang.RunnableX;
import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;

/**
Expand Down Expand Up @@ -71,4 +72,11 @@ public interface Node<RowT> extends AutoCloseable {
* Rewinds upstream.
*/
void rewind();

/**
* Schedules the given action of this execution node. If this node was closed, the task is ignored.
*
* @param task Task.
*/
void execute(RunnableX task);
}
Original file line number Diff line number Diff line change
Expand Up @@ -265,7 +265,7 @@ private void sendBatch(String nodeName, int batchId, boolean last, List<RowT> ro
ex
);

context().execute(() -> onError(wrapperEx), this::onError);
this.execute(() -> onError(wrapperEx));
});
}

Expand Down Expand Up @@ -344,7 +344,7 @@ private void flush() throws Exception {
*/
public void onNodeLeft(String nodeName) {
if (nodeName.equals(context().originatingNodeName())) {
context().execute(this::close, this::onError);
this.execute(this::close);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -130,7 +130,7 @@ public boolean isClosed() {
/** {@inheritDoc} */
@Override
public void closeInternal() {
context().execute(() -> sources().forEach(Commons::closeQuiet), this::onError);
this.execute(() -> sources().forEach(Commons::closeQuiet));
}

/** {@inheritDoc} */
Expand Down Expand Up @@ -255,7 +255,7 @@ private void exchangeBuffers() {
close();
} else if (inBuff.isEmpty() && waiting == 0) {
int req = waiting = inBufSize;
context().execute(() -> source().request(req), this::onError);
this.execute(() -> source().request(req));
}

if (!outBuff.isEmpty() || waiting == -1) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::push, this::onError);
this.execute(this::push);
}
}

Expand Down Expand Up @@ -126,7 +126,7 @@ private void push() throws Exception {

if (++processed == inBufSize && requested > 0) {
// allow others to do their job
context().execute(this::push, this::onError);
this.execute(this::push);

return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,7 +154,7 @@ public void push(RowT row) throws Exception {
if (waiting == 0 && requested > 0) {
waiting = inBufSize;

context().execute(() -> source().request(inBufSize), this::onError);
this.execute(() -> source().request(inBufSize));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ public void request(int rowsCnt) throws Exception {
if (waiting == 0) {
source().request(waiting = inBufSize);
} else if (!inLoop) {
context().execute(this::flush, this::onError);
this.execute(this::flush);
}
}

Expand Down Expand Up @@ -176,7 +176,7 @@ private void flush() throws Exception {

if (++processed >= inBufSize) {
// Allow the others to do their job.
context().execute(this::flush, this::onError);
this.execute(this::flush);

return;
}
Expand All @@ -194,7 +194,7 @@ private void flush() throws Exception {

if (++processed >= inBufSize && requested > 0) {
// allow others to do their job
context().execute(this::flush, this::onError);
this.execute(this::flush);

return;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ public void request(int rowsCnt) throws Exception {
requested = rowsCnt;

if (!inLoop) {
context().execute(this::push, this::onError);
this.execute(this::push);
}
}

Expand Down Expand Up @@ -166,7 +166,7 @@ private void push() throws Exception {
requested = 0;
downstream().end();
} else {
context().execute(this::push, this::onError);
this.execute(this::push);
}
}
}
Expand Down Expand Up @@ -228,30 +228,30 @@ public void onNext(RowT row) {
inBuffInner.add(row);

if (inBuffInner.size() == inBufSize) {
context().execute(() -> {
StorageScanNode.this.execute(() -> {
waiting = 0;
push();
}, StorageScanNode.this::onError);
});
}
}

/** {@inheritDoc} */
@Override
public void onError(Throwable throwable) {
context().execute(() -> {
StorageScanNode.this.execute(() -> {
throw throwable;
}, StorageScanNode.this::onError);
});
}

/** {@inheritDoc} */
@Override
public void onComplete() {
context().execute(() -> {
StorageScanNode.this.execute(() -> {
activeSubscription = null;
waiting = 0;

push();
}, StorageScanNode.this::onError);
});
}
}
}
Loading