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

testStatsBasedRepartitionDataOnInsert flaky #11140

Closed
losipiuk opened this issue Feb 22, 2022 · 20 comments · Fixed by #14397 or #17431
Closed

testStatsBasedRepartitionDataOnInsert flaky #11140

losipiuk opened this issue Feb 22, 2022 · 20 comments · Fixed by #14397 or #17431
Assignees
Labels
bug Something isn't working test

Comments

@losipiuk
Copy link
Member

losipiuk commented Feb 22, 2022

[ERROR] io.trino.plugin.iceberg.TestIcebergOrcConnectorTest.testStatsBasedRepartitionDataOnInsert  Time elapsed: 1.576 s  <<< FAILURE!
java.lang.AssertionError: 
[Rows for query [SELECT count(*) FROM "repartition_SELECTDISTINCTorderkeycustkeyorderstatusFROMtpchtinyordersinsert_orderstatus_1ns2vmr9lb$files"]] 
Expecting:
  <(3)>
to contain exactly in any order:
  <[(9)]>
elements not found:
  <(9)>
and elements not expected:
  <(3)>

	at io.trino.sql.query.QueryAssertions$QueryAssert.lambda$matches$2(QueryAssertions.java:346)
	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:333)
	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:328)
	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testRepartitionData(BaseIcebergConnectorTest.java:2874)
	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.lambda$testStatsBasedRepartitionData$27(BaseIcebergConnectorTest.java:2831)
	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:74)
	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:66)
	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:61)
	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionData(BaseIcebergConnectorTest.java:2830)
	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionDataOnInsert(BaseIcebergConnectorTest.java:2809)
	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
	at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
	at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
	at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
	at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
	at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
	at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
	at java.base/java.lang.Thread.run(Thread.java:829)

https://github.com/trinodb/trino/runs/5280874831?check_suite_focus=true

@fengguangyuan
Copy link

fengguangyuan commented Feb 23, 2022

Is this an occasional failure? I could not reproduce it on Master branch. :(

@ebyhr
Copy link
Member

ebyhr commented Feb 23, 2022

@fengguangyuan This is an occasional failure.

@fengguangyuan
Copy link

After a lot spare time on this issue, I think the root cause of the failure is being clear.

Basic test info:
    Table:
         name:          tpch.tiny.orders
         files:         3
         part key:      orderstatus
         part values:   ['F', 'O', 'P']
    Cluster:
         workers: 3

    Expected files count:  9

Conclusion

Imbalanced distribution of data pages on 3 output buffer slots in TaskOutputOperator leads to the unexpected number of written files, therefore the number of files could be in range [3, 9].

Details

First of all, we know that INSERT INTO statement will produce a physical plan, which is roughly described as:
scan stage(scan -> hash -> output) -> exchange stage(exchange -> exchange sink) -> write stage

At the scan stage, the last operator TaskOutputOperator is responsible for putting data pages into it's internal buffer, an instance of ArbitraryOutputBuffer which contains 3 ClientBuffers, and then waiting to be consumed by the upstream tasks in parallel, here these tasks are from exchange stage.

ArbitraryOutputBuffer has 3 output client buffers, implying 3 parallelism, because there are three simulated worker nodes created by the test framework.

Ideally, TaskOutputOperator should keep even distribution of all the data pages on 3 buffer channels, for example if there were 9 pages, each buffer should own 3.

But actually, considering all the data pages are not received by ArbitraryOutputBuffer once, ArbitraryOutputBuffer using an improved round-robin strategy to locate buffers for each enqueue. The code below describes the strategy and is what I used to test:

public class ArbitraryOutputBuffer
        implements OutputBuffer
{
    @GuardedBy("this")
    private OutputBuffers outputBuffers = createInitialEmptyOutputBuffers(ARBITRARY);

    // all the pages should be put into this buffer at first, then move to `buffers`
    private final MasterBuffer masterBuffer;

    // the size is determined by the upstream task, here it's 3 corresponding to 3 worker nodes
    @GuardedBy("this")
    private final ConcurrentMap<OutputBufferId, ClientBuffer> buffers = new ConcurrentHashMap<>();

    //  The index of the first client buffer that should be polled
    private final AtomicInteger nextClientBufferIndex = new AtomicInteger(0);

    @Override
    public void enqueue(List<Slice> pages)
    {
        ...
        /* Test code start */
        long beforeCnt = masterBuffer.getBufferedPages();
        /* Test code end */

        // add pages to the buffer (this will increase the reference count by one)
        masterBuffer.addPages(serializedPageReferences);
        
        /* Test code start */
        System.out.printf("ThreadId %s: page size: %s, before master buffer size %s, after master buffer size %s,\n",
                Thread.currentThread().getName(), bytesAdded, beforeCnt, masterBuffer.getBufferedPages());
        /* Test code end */

        // process any pending reads from the client buffers
        List<ClientBuffer> buffers = safeGetBuffersSnapshot();
        if (buffers.isEmpty()) {
            return;
        }
        // handle potential for racy update of next index and client buffers present
        int index = nextClientBufferIndex.get() % buffers.size();
        for (int i = 0; i < buffers.size(); i++) {
            // buffers.get(index).loadPagesIfNecessary(masterBuffer);
            
            /* Test code start */
            ClientBuffer clientBuffer = buffers.get(index);
            clientBuffer.loadPagesIfNecessary(masterBuffer);
            BufferInfo bufferInfo = clientBuffer.getInfo();
            System.out.printf("ThreadId %s: %s now buffered total %s pages on bufferId %s with destroyed %s!\n",
                    Thread.currentThread().getName() + "-" + Thread.currentThread().getId(),
                    this.getClass().getName(),
                    bufferInfo.getBufferedPages(),
                    bufferInfo.getBufferId(),
                    clientBuffer.isDestroyed());
            /* Test code end */

            index = (index + 1) % buffers.size();

            if (masterBuffer.isEmpty()) {
                // Resume from the next client buffer on the next iteration
                nextClientBufferIndex.set(index);
                break;
            }
        }
    }
}

Below logs are from task 20220308_114754_00170_7cd8a.2.1.0 at scan stage, we could see that buffer-1 received the most pages, buffer-2 received fewer pages, while buffer-0 received 0. With no doubts, all the tasks consumed pages from buffer-0 will produce nothing, that is the exchange task reading buffer-0 sink nothing, and finally the write task reading sinked data partition 0 wrote nothing.

This tells the reason why the test could occasionally failed on mismatched files count. In this case, total 6 files were produced by two workers, rather than expected 9.

ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: page size: 5501, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: page size: 5522, before master buffer size 2, after master buffer size 3,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: page size: 2540, before master buffer size 3, after master buffer size 4,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-3-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: page size: 5501, before master buffer size 4, after master buffer size 5,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: page size: 5501, before master buffer size 1, after master buffer size 2,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: page size: 5522, before master buffer size 6, after master buffer size 7,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: page size: 5501, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: page size: 2540, before master buffer size 7, after master buffer size 8,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: page size: 5522, before master buffer size 5, after master buffer size 6,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-0-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 8 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: page size: 5522, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: page size: 2540, before master buffer size 1, after master buffer size 2,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-1-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 8 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 8 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: page size: 2540, before master buffer size 2, after master buffer size 3,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-2-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 8 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: page size: 5501, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 3 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: page size: 5522, before master buffer size 1, after master buffer size 2,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: page size: 5501, before master buffer size 2, after master buffer size 3,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 2 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: page size: 5522, before master buffer size 3, after master buffer size 4,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 2 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: page size: 2540, before master buffer size 4, after master buffer size 5,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-6-66: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: page size: 5501, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: page size: 5501, before master buffer size 1, after master buffer size 2,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: page size: 2540, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-4-63: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: page size: 5522, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: page size: 2540, before master buffer size 1, after master buffer size 2,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-7-64: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: page size: 5522, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 2 pages on bufferId 2 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: page size: 2540, before master buffer size 0, after master buffer size 1,
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 0 pages on bufferId 0 with destroyed false!
ThreadId 20220308_114754_00170_7cd8a.2.1.0-5-65: io.trino.execution.buffer.ArbitraryOutputBuffer now buffered total 1 pages on bufferId 1 with destroyed false!

Discussion

  1. Imbalanced distribution of pages by ArbitraryOutputBuffer is acceptable?
  2. Imbalanced distribution will decrease the overall performance?
  3. It's better to use a more heuristic method to determine the output buffer id, such as the client buffer size, the upstream workers QPS or ... ?

I hope the above explanations would not make you puzzled. :)

@fengguangyuan
Copy link

@ebyhr @losipiuk

@findepi
Copy link
Member

findepi commented Mar 9, 2022

This tells the reason why the test could occasionally failed on mismatched files count. In this case, total 6 files were produced by two workers, rather than expected 9.

Correct. That's why the test uses assertEventually.
Apparently that's sometimes not enough.

@findepi
Copy link
Member

findepi commented May 30, 2022

@findepi
Copy link
Member

findepi commented Jun 20, 2022

@findepi
Copy link
Member

findepi commented Jun 28, 2022

https://github.com/trinodb/trino/runs/7089990501?check_suite_focus=true

Error:    TestIcebergParquetConnectorTest>BaseIcebergConnectorTest.testStatsBasedRepartitionDataOnInsert:3173->BaseIcebergConnectorTest.testStatsBasedRepartitionData:3195->BaseIcebergConnectorTest.lambda$testStatsBasedRepartitionData$15:3196->BaseIcebergConnectorTest.testRepartitionData:3239 [Rows for query [SELECT count(*) FROM "repartition_SELECTDISTINCTorderkeycustkeyorderstatusFROMtpchtinyordersinsert_orderstatus_ybf5539lvf$files"]] 
Expecting:
  <(6)>
to contain exactly in any order:
  <[(9)]>
elements not found:
  <(9)>
and elements not expected:
  <(6)>

@findepi
Copy link
Member

findepi commented Sep 27, 2022

@findinpath
Copy link
Contributor

findinpath commented Oct 1, 2022

@findinpath
Copy link
Contributor

@findepi
Copy link
Member

findepi commented May 8, 2023

@wendigo any comment why this is reopened? thanks

@wendigo
Copy link
Contributor

wendigo commented May 8, 2023

I've linked a new occurance

@findepi
Copy link
Member

findepi commented May 8, 2023

i didn't see any comment here, hence the question.

@wendigo
Copy link
Contributor

wendigo commented May 8, 2023

1 similar comment
@wendigo
Copy link
Contributor

wendigo commented May 8, 2023

@wendigo
Copy link
Contributor

wendigo commented May 8, 2023

@findepi
Copy link
Member

findepi commented May 9, 2023

https://github.com/trinodb/trino/actions/runs/4925370678/jobs/8800359681?pr=17383

2023-05-09T14:45:50.7113079Z [ERROR] Tests run: 1556, Failures: 1, Errors: 0, Skipped: 31, Time elapsed: 1,855.731 s <<< FAILURE! - in TestSuite
2023-05-09T14:45:50.7114300Z [ERROR] io.trino.plugin.iceberg.TestIcebergMinioOrcConnectorTest.testStatsBasedRepartitionDataOnInsert  Time elapsed: 348.707 s  <<< FAILURE!
2023-05-09T14:45:50.7115313Z org.assertj.core.error.MultipleAssertionsError: 
2023-05-09T14:45:50.7115625Z 
2023-05-09T14:45:50.7115803Z The following assertion failed:
2023-05-09T14:45:50.7116544Z 1) [Rows for query [SELECT count(*) FROM "repartition_SELECTDISTINCTorderkeycustkeyorderstatusFROMtpchtinyordersinsert_orderstatus_qsytnlfkk0$files"]] 
2023-05-09T14:45:50.7117203Z Expecting actual:
2023-05-09T14:45:50.7117537Z   (6)
2023-05-09T14:45:50.7117890Z to contain exactly in any order:
2023-05-09T14:45:50.7118241Z   [(9)]
2023-05-09T14:45:50.7118563Z elements not found:
2023-05-09T14:45:50.7118895Z   (9)
2023-05-09T14:45:50.7119231Z and elements not expected:
2023-05-09T14:45:50.7119577Z   (6)
2023-05-09T14:45:50.7119751Z 
2023-05-09T14:45:50.7119758Z 
2023-05-09T14:45:50.7120192Z 	at org.assertj.core.error.AssertionErrorCreator.multipleAssertionsError(AssertionErrorCreator.java:106)
2023-05-09T14:45:50.7121100Z 	at org.assertj.core.api.AbstractAssert.multipleAssertionsError(AbstractAssert.java:979)
2023-05-09T14:45:50.7121844Z 	at org.assertj.core.api.AbstractAssert.satisfiesForProxy(AbstractAssert.java:875)
2023-05-09T14:45:50.7122512Z 	at org.assertj.core.api.AbstractAssert.satisfies(AbstractAssert.java:862)
2023-05-09T14:45:50.7123271Z 	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:383)
2023-05-09T14:45:50.7124013Z 	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:377)
2023-05-09T14:45:50.7124766Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testRepartitionData(BaseIcebergConnectorTest.java:4637)
2023-05-09T14:45:50.7125663Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.lambda$testStatsBasedRepartitionData$23(BaseIcebergConnectorTest.java:4594)
2023-05-09T14:45:50.7126418Z 	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:43)
2023-05-09T14:45:50.7127046Z 	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:34)
2023-05-09T14:45:50.7127954Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionData(BaseIcebergConnectorTest.java:4593)
2023-05-09T14:45:50.7128970Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionDataOnInsert(BaseIcebergConnectorTest.java:4574)
2023-05-09T14:45:50.7129911Z 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
2023-05-09T14:45:50.7130667Z 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
2023-05-09T14:45:50.7131479Z 	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
2023-05-09T14:45:50.7132156Z 	at java.base/java.lang.reflect.Method.invoke(Method.java:568)
2023-05-09T14:45:50.7132955Z 	at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
2023-05-09T14:45:50.7133638Z 	at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
2023-05-09T14:45:50.7134292Z 	at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
2023-05-09T14:45:50.7134905Z 	at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
2023-05-09T14:45:50.7135568Z 	at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
2023-05-09T14:45:50.7136221Z 	at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
2023-05-09T14:45:50.7136986Z 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
2023-05-09T14:45:50.7137808Z 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
2023-05-09T14:45:50.7138371Z 	at java.base/java.lang.Thread.run(Thread.java:833)

@krvikash
Copy link
Contributor

krvikash commented May 9, 2023

https://github.com/trinodb/trino/actions/runs/4926424212/jobs/8805991023?pr=17409

2023-05-09T19:07:40.0167702Z [ERROR] Tests run: 1558, Failures: 1, Errors: 0, Skipped: 31, Time elapsed: 2,078.153 s <<< FAILURE! - in TestSuite
2023-05-09T19:07:40.0169427Z [ERROR] io.trino.plugin.iceberg.TestIcebergAvroConnectorTest.testStatsBasedRepartitionDataOnInsert  Time elapsed: 0.019 s  <<< FAILURE!
2023-05-09T19:07:40.0170346Z org.assertj.core.error.MultipleAssertionsError: 
2023-05-09T19:07:40.0170834Z 
2023-05-09T19:07:40.0171079Z The following assertion failed:
2023-05-09T19:07:40.0172307Z 1) [Rows for query [SELECT count(*) FROM "repartition_SELECTDISTINCTorderkeycustkeyorderstatusFROMtpchtinyordersinsert_orderstatus_bal0f49dd6$files"]] 
2023-05-09T19:07:40.0172970Z Expecting actual:
2023-05-09T19:07:40.0173280Z   (3)
2023-05-09T19:07:40.0173636Z to contain exactly in any order:
2023-05-09T19:07:40.0173993Z   [(9)]
2023-05-09T19:07:40.0174318Z elements not found:
2023-05-09T19:07:40.0174639Z   (9)
2023-05-09T19:07:40.0174977Z and elements not expected:
2023-05-09T19:07:40.0175299Z   (3)
2023-05-09T19:07:40.0175516Z 
2023-05-09T19:07:40.0175615Z 
2023-05-09T19:07:40.0176128Z 	at org.assertj.core.error.AssertionErrorCreator.multipleAssertionsError(AssertionErrorCreator.java:106)
2023-05-09T19:07:40.0177046Z 	at org.assertj.core.api.AbstractAssert.multipleAssertionsError(AbstractAssert.java:979)
2023-05-09T19:07:40.0177932Z 	at org.assertj.core.api.AbstractAssert.satisfiesForProxy(AbstractAssert.java:875)
2023-05-09T19:07:40.0178770Z 	at org.assertj.core.api.AbstractAssert.satisfies(AbstractAssert.java:862)
2023-05-09T19:07:40.0179463Z 	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:383)
2023-05-09T19:07:40.0180141Z 	at io.trino.sql.query.QueryAssertions$QueryAssert.matches(QueryAssertions.java:377)
2023-05-09T19:07:40.0181071Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testRepartitionData(BaseIcebergConnectorTest.java:4637)
2023-05-09T19:07:40.0182040Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.lambda$testStatsBasedRepartitionData$23(BaseIcebergConnectorTest.java:4594)
2023-05-09T19:07:40.0182850Z 	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:43)
2023-05-09T19:07:40.0183527Z 	at io.trino.testing.assertions.Assert.assertEventually(Assert.java:34)
2023-05-09T19:07:40.0184390Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionData(BaseIcebergConnectorTest.java:4593)
2023-05-09T19:07:40.0185950Z 	at io.trino.plugin.iceberg.BaseIcebergConnectorTest.testStatsBasedRepartitionDataOnInsert(BaseIcebergConnectorTest.java:4574)
2023-05-09T19:07:40.0186889Z 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
2023-05-09T19:07:40.0187674Z 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
2023-05-09T19:07:40.0188664Z 	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
2023-05-09T19:07:40.0189393Z 	at java.base/java.lang.reflect.Method.invoke(Method.java:568)
2023-05-09T19:07:40.0190087Z 	at org.testng.internal.MethodInvocationHelper.invokeMethod(MethodInvocationHelper.java:104)
2023-05-09T19:07:40.0190805Z 	at org.testng.internal.Invoker.invokeMethod(Invoker.java:645)
2023-05-09T19:07:40.0191408Z 	at org.testng.internal.Invoker.invokeTestMethod(Invoker.java:851)
2023-05-09T19:07:40.0192711Z 	at org.testng.internal.Invoker.invokeTestMethods(Invoker.java:1177)
2023-05-09T19:07:40.0198369Z 	at org.testng.internal.TestMethodWorker.invokeTestMethods(TestMethodWorker.java:129)
2023-05-09T19:07:40.0199050Z 	at org.testng.internal.TestMethodWorker.run(TestMethodWorker.java:112)
2023-05-09T19:07:40.0199697Z 	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
2023-05-09T19:07:40.0200413Z 	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
2023-05-09T19:07:40.0200926Z 	at java.base/java.lang.Thread.run(Thread.java:833)

@findepi
Copy link
Member

findepi commented May 10, 2023

https://github.com/trinodb/trino/actions/runs/4927265886/jobs/8807331960?pr=17383

2023-05-09T22:35:32.9270291Z [ERROR] Tests run: 1556, Failures: 1, Errors: 0, Skipped: 31, Time elapsed: 2,123.722 s <<< FAILURE! - in TestSuite
2023-05-09T22:35:32.9273524Z [ERROR] io.trino.plugin.iceberg.TestIcebergMinioOrcConnectorTest.testStatsBasedRepartitionDataOnInsert  Time elapsed: 14.599 s  <<< FAILURE!
2023-05-09T22:35:32.9274503Z org.assertj.core.error.MultipleAssertionsError: 
2023-05-09T22:35:32.9274866Z 
2023-05-09T22:35:32.9275082Z The following assertion failed:
2023-05-09T22:35:32.9275910Z 1) [Rows for query [SELECT count(*) FROM "repartition_SELECTDISTINCTorderkeycustkeyorderstatusFROMtpchtinyordersinsert_orderstatus_j4vhyeg40j$files"]] 
2023-05-09T22:35:32.9276660Z Expecting actual:
2023-05-09T22:35:32.9277177Z   (6)
2023-05-09T22:35:32.9277617Z to contain exactly in any order:
2023-05-09T22:35:32.9278052Z   [(9)]
2023-05-09T22:35:32.9278456Z elements not found:
2023-05-09T22:35:32.9278862Z   (9)
2023-05-09T22:35:32.9279294Z and elements not expected:
2023-05-09T22:35:32.9279688Z   (6)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
bug Something isn't working test
7 participants