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

sql: call DrainMeta and Next from a single goroutine #48785

Closed
asubiotto opened this issue May 13, 2020 · 17 comments · Fixed by #50009
Closed

sql: call DrainMeta and Next from a single goroutine #48785

asubiotto opened this issue May 13, 2020 · 17 comments · Fixed by #50009
Assignees
Labels
A-sql-execution Relating to SQL execution. C-test-failure Broken test (automatically or manually discovered).

Comments

@asubiotto
Copy link
Contributor

The inner-join logictest fails under stress with the fakedist configuration. Repro:

make stress PKG=./pkg/sql/logictest TESTS="TestLogic//inner-join" TESTFLAGS="-v -config=fakedist"

Looks like sometimes rows are being only partially returned:

                testdata/logic_test/inner-join:69: SELECT a, b, c FROM abc_decimal WHERE EXISTS (SELECT * FROM def_decimal WHERE a::string=d::string)
                expected:
                    1     1     1
                    1     1     1
                    1.0   1.0   1.0
                    1.00  1.00  1.00
                but found (query options: "rowsort" -> ignore the following ordering of rows) :
                    1.00  1.00  1.00
@asubiotto asubiotto self-assigned this May 13, 2020
@blathers-crl
Copy link

blathers-crl bot commented May 13, 2020

Hi @asubiotto, please add a C-ategory label to your issue. Check out the label system docs.

While you're here, please consider adding an A- label to help keep our repository tidy.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is otan.

@asubiotto asubiotto added A-sql-execution Relating to SQL execution. C-test-failure Broken test (automatically or manually discovered). labels May 13, 2020
@knz
Copy link
Contributor

knz commented May 13, 2020

@asubiotto
Copy link
Contributor Author

@yuzefovich based on a git bisect it looks like #48668 is the first bad commit. Would you be able to look into this?

@yuzefovich
Copy link
Member

Sure.

@yuzefovich yuzefovich assigned yuzefovich and unassigned asubiotto May 13, 2020
@yuzefovich
Copy link
Member

This issue is quite puzzling. I've been playing around with this slightly modified logic test:

# LogicTest: 5node

statement ok
CREATE TABLE abc_decimal (i INT PRIMARY KEY, a DECIMAL, b DECIMAL, c DECIMAL);
INSERT INTO abc_decimal VALUES (1, 1, 1, 1), (2, 1, 1, 1), (3, 1.0, 1.0, 1.0), (4, 1.00, 1.00, 1.00);
ALTER TABLE abc_decimal SPLIT AT VALUES (2), (3), (4);
ALTER TABLE abc_decimal EXPERIMENTAL_RELOCATE SELECT ARRAY[i+1], i FROM generate_series(1, 4) as g(i)

statement ok
CREATE TABLE def_decimal (i INT PRIMARY KEY, d DECIMAL, e DECIMAL, f DECIMAL);
INSERT INTO def_decimal VALUES (1, 1, 1, 1), (2, 1.0, 1.0, 1.0), (3, 1.00, 1.00, 1.00);
ALTER TABLE def_decimal SPLIT AT VALUES (2), (3);
ALTER TABLE def_decimal EXPERIMENTAL_RELOCATE SELECT ARRAY[i+1], i FROM generate_series(1, 3) as g(i)

query RRR rowsort
SELECT a, b, c FROM abc_decimal WHERE EXISTS (SELECT * FROM def_decimal WHERE a::string=d::string)
----
1     1     1
1     1     1
1.0   1.0   1.0
1.00  1.00  1.00

and what's interesting is that the failure seems to occur only if both tables do not have any data on the gateway node (that's the reason for i+1 when relocating).

I feel like the problem might be with parallel unordered synchronizer, and that PR to short-circuit the hash joiner just exposed it (namely, made it more likely to occur on this logic test). I think so because if it was the hash joiner to blame, then I'd expect the failure to occur every time, but it occurs quite rarely and only under stress which indicates that there some race going on.

@asubiotto asubiotto moved this from Triage to 20.2 B in BACKLOG, NO NEW ISSUES: SQL Execution May 14, 2020
@yuzefovich
Copy link
Member

I stared at it for another hour or two but didn't make any progress.

Another logic test that be used to repro is

# LogicTest: fakedist

statement ok
CREATE TABLE twocolumn (x INT, y INT PRIMARY KEY);
INSERT INTO twocolumn(x, y) VALUES (44,51), (45,45)

query IIIIII rowsort
SELECT a.x AS s, b.x, c.x, a.y, b.y, c.y FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c USING(x))
----
44  44  44  51  51  51
45  45  45  45  45  45

and I wasn't successful in converting it into using 5node config with ranges relocation.

My theory still remains the same - that somehow parallel unordered synchronizer is to blame. @asubiotto do you mind taking a look?

@asubiotto
Copy link
Contributor Author

Sure.

@asubiotto asubiotto assigned asubiotto and unassigned yuzefovich May 15, 2020
@jordanlewis
Copy link
Member

------- Stdout: -------
=== RUN   TestLogic/fakedist/subquery_correlated
=== PAUSE TestLogic/fakedist/subquery_correlated
=== CONT  TestLogic/fakedist/subquery_correlated
        --- FAIL: TestLogic/fakedist/subquery_correlated (5.03s)
            logic.go:1952: 
                 
                testdata/logic_test/subquery_correlated:84: SELECT *
                FROM c
                WHERE
                    'CA' IN (SELECT ship FROM o WHERE o.c_id=c.c_id)
                    AND 'TX' NOT IN (SELECT ship FROM o WHERE o.c_id=c.c_id);
                expected:
                    1  CA
                    
                but found (query options: "rowsort" -> ignore the following ordering of rows) :
                    1  CA
                    2  TX
                    
            logic.go:2530: 
                testdata/logic_test/subquery_correlated:93: error while processing
            logic.go:2530: testdata/logic_test/subquery_correlated:93: too many errors encountered, skipping the rest of the input

I just got this failure on a CI run. Is it related?

@yuzefovich
Copy link
Member

Yeah, I think it's the same underlying problem.

@andreimatei
Copy link
Contributor

In case it helps anything, I got

=== CONT  TestLogic/fakedist/join
rewrote:
CREATE TABLE abc (a INT8, b INT8, c INT8, PRIMARY KEY (a, c), FAMILY (a), FAMILY (b, c));
INSERT INTO abc VALUES (1, 1, 2), (2, 1, 1), (2, NULL, 2);


        --- FAIL: TestLogic/fakedist/join (45.88s)
            logic.go:1952: 
                 
                testdata/logic_test/join:429: SELECT * FROM (twocolumn AS a JOIN twocolumn AS b USING(x) JOIN twocolumn AS c USING(x)) ORDER BY x LIMIT 1
                expected:
                    x  y  y  y
                    42 53 53 53
                    
                but found (query options: "colnames") :
                    x   y   y   y
                    44  51  51  51

@asubiotto
Copy link
Contributor Author

Forcing the use of a serial synchronizer (slower but simpler logic):

                } else {
-                       if opt == flowinfra.FuseAggressively {
+                       if true || opt == flowinfra.FuseAggressively {
                                op = colexec.NewSerialUnorderedSynchronizer(inputStreamOps, input.ColumnTypes)
                        } else {
                                op = colexec.NewParallelUnorderedSynchronizer(inputStreamOps, input.ColumnTypes, s.waitGroup)

still reproduces this problem so I don't think this is a parallel unordered synchronizer problem, and as soon as I comment out the short-circuiting logic in the hash joiner, this problem doesn't reproduce. Another change that makes this test pass is exhausting the left side:

                                if hj.spec.joinType == sqlbase.JoinType_INNER ||
                                        hj.spec.joinType == sqlbase.JoinType_RIGHT_OUTER ||
                                        hj.spec.joinType == sqlbase.JoinType_LEFT_SEMI {
+                                       for {
+                                               b := hj.inputOne.Next(ctx)
+                                               if b.Length() == 0 {
+                                                       break
+                                               }
+                                       }

Which indicates that not calling Next on the left input is having undesired side effects (which should not happen). @yuzefovich does anything come to mind? Will keep on digging.

@yuzefovich
Copy link
Member

Hm, since we started populating batches of projecting operators upfront, we're able to short-circuit execution of operators when they receive zero-length batch from its inputs, however, in all previous scenarios we would always call Next on the inputs at least once, and this change in the hash joiner seems to be the first precedent of not doing so, but I can't think of anything that would break because of it - Init is still called as required by the interface.

I'm thinking that #49147 that shows a data race when cleaning up disk spilling infrastructure might be the root cause of this issue as well, but I haven't looked too closely into that issue.

@asubiotto
Copy link
Contributor Author

I'm thinking that #49147 that shows a data race when cleaning up disk spilling infrastructure might be the root cause of this issue as well, but I haven't looked too closely into that issue.

I took a look at that and fixed the race and it doesn't make a difference for the erroneous test output.

@asubiotto
Copy link
Contributor Author

Going to merge the single call to Next to fix the flake while I investigate further

craig bot pushed a commit that referenced this issue May 19, 2020
49202: colexec: fix race and TestLogic/fakedist/inner-join flake r=yuzefovich a=asubiotto

Look at commits for details

Release note: None

Addresses #48785 
Fixes #49147 

Co-authored-by: Alfonso Subiotto Marques <alfonso@cockroachlabs.com>
@asubiotto
Copy link
Contributor Author

asubiotto commented May 19, 2020

Got to the bottom of this. This is the EXPLAIN (DISTSQL) plan for the repro shared in #48785 (comment)

image

In the repro, it looks like HashJoiner/9 only reads 1 row from the left side but 2 rows from the right side. It therefore only emits one result tuple when it should be emitting 2. This is cause by a missing row on the left side.

This row can be traced back to the inbox pointed out on the diagram. The row is correctly sent over the wire but DrainMeta is called before Next, so the Inbox returns zero rows. This is correct Inbox behavior, the problem is that DrainMeta is called prematurely.

HashJoiner/7 short circuits because it gets zero rows from its right side. The downstream outbox gets a zero length batch and therefore calls DrainMeta on its metadata sources, which includes the Inbox. This Inbox is in the outbox's input graph, but the problem is that we assume that the input graph is drained of rows when the outbox receives a zero-length batch. This is not the case with short-circuiting hash joiners, since the left input could be a hash router that still needs to send rows to another node. Calling Next once on the left side fixes this issue because it wait until the HashRouter has finished pushing all rows.

To properly fix this, I think we need to make the outbox call DrainMeta on its upstream HashRouter. The HashRouter would only cancel operation once all outputs have called DrainMeta.

@yuzefovich
Copy link
Member

Nice find!

@knz
Copy link
Contributor

knz commented May 19, 2020

That is a very nice find indeed.

@asubiotto asubiotto changed the title sql: TestLogic/fakedist/inner-join failure sql: call DrainMeta and Next from a single goroutine May 19, 2020
@asubiotto asubiotto moved this from 20.2 B to 20.2 C in BACKLOG, NO NEW ISSUES: SQL Execution Jun 2, 2020
@craig craig bot closed this as completed in e195958 Jun 16, 2020
@asubiotto asubiotto moved this from 20.2 C to 20.2 C DONE in BACKLOG, NO NEW ISSUES: SQL Execution Jun 16, 2020
craig bot pushed a commit that referenced this issue Jul 28, 2020
50922: importccl: support `unique_rowid()` as default expression for IMPORT INTO r=Anzoteh96 a=Anzoteh96

The PR #50295 supports non-targeted columns with constant expression. This PR is a follow up to that in adding support to `unique_rowid()`.

Previously, the only support given to `rowid` as a default expression is for hidden column, which is a function of timestamp, row number, and source ID (the ID of processor). To accommodate for more usage of `unique_rowid()`, this PR modifies the `unique_rowid` function by making `unique_rowid` as a function of: 
1. timestamp; 
2. row number; 
3. source ID; 
4. the total occurrences of `unique_rowid` in the table schema; 
5. instances of each `unique_rowid` within each row. 

In addition, this PR also modifies the visitor method #51390 by adding override methods for volatile methods like `unique_rowid`. Annotations containing the total occurrences of `unique_rowid` and `unique_rowid` instances within a row are stored inside `evalCtx`, which will be read and updated when visitor walks through the default expression at the sanitization stage, and when default expression is evaluated at each row. 

Partially addresses #48253 

Release note (general change): IMPORT INTO now supports `unique_rowid()` as a default expression.

51518: rowflow,colexec: make routers propagate errors to all non-closed outputs r=yuzefovich a=yuzefovich

This commit changes the way we propagate the errors in the hash router
so that the error metadata is sent on all non-closed streams.
Previously, we would be sending it over only the first non-closed stream
which could result in the processors on the same stage as that single
stream end to treat the absence of rows and errors as the input being
exhausted successfully, which is wrong because the input did encounter
an error.

The same thing has been happening in the vectorized flow, but in that
case the problem is less severe - the issue will present itself only
when we have wrapped processors (because the materializers will prevent
the propagation throughout the whole flow as described below):
In the vectorized engine we use panic-catch mechanism of error
propagation, and we end up with the following sequence of events:
1. an operator encounters an error on any node (e.g. `colBatchScan`
encounters RWUI error on a remote node). It is not an internal vectorized
error, so the operator will panic with `colexecerror.ExpectedError`.
2. the panic is caught by one of the catchers (it can be a parallel
unordered synchronizer goroutine, an outbox goroutine, a materializer,
a hash router)
3. that component will then decide how to propagate the error further:
3.1 if it is a parallel unordered synchronizer, then it will cancel all
of its inputs and will repanic
3.2 if it is an outbox, the error is sent as metadata which will be
received by an inbox which will panic with it
3.3. if it is a materializer, then it might swallow the error (this is
the reason we need for the vectorized hash router to send the error to
all of its inputs). The swallowing is acceptable if it is the root
materializer though.
3.4 if it is a hash router, it'll cancel all of its outputs and will
forward the error on each of the outputs.

Fixes: #51458.

Release note (bug fix): Previously, CockroachDB could return incorrect
results on query that encountered ReadWithinUncertaintyInterval error,
and this has been fixed.

52016: colexec: re-enable short-circuiting in the hash joiner r=yuzefovich a=yuzefovich

This commit re-enables short-circuiting logic in the hash joiner when
the build side is empty (it was temporarily disabled because of #48785
which has been fixed).

Fixes: #49631.

Release note: None

52027: sql: skip TestQueryProgress r=yuzefovich a=yuzefovich

This test started failing more often, so we'll skip it temporarily until
we figure it out.

Addresses: #51356.

Release note: None

Co-authored-by: anzoteh96 <anzot@cockroachlabs.com>
Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-sql-execution Relating to SQL execution. C-test-failure Broken test (automatically or manually discovered).
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants