szehon-ho commented on code in PR #7933: URL: https://github.com/apache/iceberg/pull/7933#discussion_r1258662864
########## core/src/test/java/org/apache/iceberg/actions/TestCommitService.java: ########## @@ -0,0 +1,112 @@ +/* + * 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.iceberg.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestCommitService extends TableTestBase { + + public TestCommitService() { + super(1); + } + + @Test + public void testCommittedResultsCorrectly() { + CustomCommitService commitService = new CustomCommitService(table, 5, 10000); + commitService.start(); + + ExecutorService executorService = Executors.newFixedThreadPool(10); + Tasks.range(100).executeWith(executorService).run(commitService::offer); + commitService.close(); + + Set<Integer> expected = Sets.newHashSet(IntStream.range(0, 100).iterator()); + Set<Integer> actual = Sets.newHashSet(commitService.results()); + Assertions.assertThat(actual).isEqualTo(expected); + } + + @Test + public void testAbortFileGroupsAfterTimeout() { + CustomCommitService commitService = new CustomCommitService(table, 5, 200); + commitService.start(); + + // Add the number of less than rewritesPerCommit + for (int i = 0; i < 4; i++) { + commitService.offer(i); + } + + // Simulate the latest group of rewrite + CustomCommitService spyCommitService = spy(commitService); + doReturn(false).when(spyCommitService).canCreateCommitGroup(); + for (int i = 4; i < 8; i++) { Review Comment: > However, the committerService is not stopped and it still could commit the file groups (5 to 9). Is it true? I dont see the group 5-9 will ever proceed because we always return false for canCreateCommitGroup(). (also ran the test to check). Am I missing something? I thought race condition is between commit or aborting 0-4 group. If that is true, how about adding these comments? ``` // Add file groups 0-4 for commit. // There are less than the rewritesPerCommit, and thus commitService will not commit until next batch added. // During commit, these wiil sleep a fixed duration, to test race condition with close abort (below) for (int i = 0; i < 4; i++) { commitService.offer(i); } // Add file groups 5-9 for commit // These are gated to not be able to commit, // and only serve to allow file 0-4 to proceed CustomCommitService spyCommitService = spy(commitService); doReturn(false).when(spyCommitService).canCreateCommitGroup(); for (int i = 4; i < 8; i++) { spyCommitService.offer(i); } // close commitService. // This will wait a fixed duration to abort file group 0-4 // testing race condition as they are attempt to finish commit (above) Assertions.assertThatThrownBy(commitService::close) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Timeout occurred when waiting for commits"); // Wait for the commitService to finish trying to commit file groups Awaitility.await() .atMost(5, TimeUnit.SECONDS) .pollInSameThread() .untilAsserted(() -> assertThat(commitService.completedRewritesAllCommitted()).isTrue()); Assertions.assertThat(commitService.results()) .doesNotContainAnyElementsOf(commitService.aborted); Assertions.assertThat(commitService.results()).isEqualTo(ImmutableList.of(0, 1, 2, 3, 4)); ``` Can you check if comments and my understanding is right? I also feel the commitService results will not always have 0-4 then? But rather it should either be aborted or committed. ########## core/src/test/java/org/apache/iceberg/actions/TestCommitService.java: ########## @@ -0,0 +1,113 @@ +/* + * 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.iceberg.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Tasks; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestCommitService extends TableTestBase { + + public TestCommitService() { + super(1); + } + + @Test + public void testCommittedResultsCorrectly() { + CustomCommitService commitService = new CustomCommitService(table, 5, 10000); + commitService.start(); + + ExecutorService executorService = Executors.newFixedThreadPool(10); + int numberOfFileGroups = 100; + Tasks.range(numberOfFileGroups).executeWith(executorService).run(commitService::offer); + commitService.close(); + + Set<Integer> expected = Sets.newHashSet(IntStream.range(0, 100).iterator()); + Set<Integer> actual = Sets.newHashSet(commitService.results()); + Assertions.assertThat(actual).isEqualTo(expected); + } + + @Test + public void testAbortFileGroupsAfterTimeout() { + CustomCommitService commitService = new CustomCommitService(table, 5, 200); + commitService.start(); + + // Add the number of less than rewritesPerCommit + for (int i = 0; i < 4; i++) { + commitService.offer(i); + } + + // Simulate the last group of rewrite + CustomCommitService spyCommitService = spy(commitService); + doReturn(false).when(spyCommitService).canCreateCommitGroup(); + for (int i = 4; i < 8; i++) { + spyCommitService.offer(i); + } + + Assertions.assertThatThrownBy(commitService::close) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Timeout occurred when waiting for commits"); + + // Wait for the committerService finish commit the remaining file groups Review Comment: Typo, also suggested the following comment (see my other review comment for context) // Wait for the commitService to finish trying to commit file groups -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
