This is an automated email from the ASF dual-hosted git repository.

arvid pushed a commit to branch release-1.14
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 881b72d1db2a30ddfd570719e549f5122d3d164a
Author: Qingsheng Ren <[email protected]>
AuthorDate: Mon Aug 23 17:49:14 2021 +0800

    [FLINK-23914][connector/testing-framework] Improve descriptions in test 
data matchers to reveal more info on failure
    
    (cherry picked from commit aadcfbb001021d7b76bb00578199bdd760b8fb2b)
---
 .../test/common/utils/TestDataMatchers.java        | 148 +++++++++++++--------
 .../test/common/utils/IteratorWithCurrentTest.java |  77 -----------
 .../test/common/utils/TestDataMatchersTest.java    |  71 ++++++++--
 3 files changed, 155 insertions(+), 141 deletions(-)

diff --git 
a/flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/utils/TestDataMatchers.java
 
b/flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/utils/TestDataMatchers.java
index 19e3478..55e0c2f 100644
--- 
a/flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/utils/TestDataMatchers.java
+++ 
b/flink-test-utils-parent/flink-connector-testing/src/main/java/org/apache/flink/connectors/test/common/utils/TestDataMatchers.java
@@ -26,7 +26,6 @@ import org.hamcrest.TypeSafeDiagnosingMatcher;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.NoSuchElementException;
 
 /** Matchers for validating test data. */
 @Internal
@@ -38,12 +37,12 @@ public class TestDataMatchers {
         return new MultipleSplitDataMatcher<>(testRecordsLists);
     }
 
-    public static <T> SplitDataMatcher<T> matchesSplitTestData(List<T> 
testData) {
-        return new SplitDataMatcher<>(testData);
+    public static <T> SingleSplitDataMatcher<T> matchesSplitTestData(List<T> 
testData) {
+        return new SingleSplitDataMatcher<>(testData);
     }
 
-    public static <T> SplitDataMatcher<T> matchesSplitTestData(List<T> 
testData, int limit) {
-        return new SplitDataMatcher<>(testData, limit);
+    public static <T> SingleSplitDataMatcher<T> matchesSplitTestData(List<T> 
testData, int limit) {
+        return new SingleSplitDataMatcher<>(testData, limit);
     }
 
     // ---------------------------- Matcher Definitions 
--------------------------------
@@ -53,7 +52,7 @@ public class TestDataMatchers {
      *
      * @param <T> Type of validating record
      */
-    public static class SplitDataMatcher<T> extends 
TypeSafeDiagnosingMatcher<Iterator<T>> {
+    public static class SingleSplitDataMatcher<T> extends 
TypeSafeDiagnosingMatcher<Iterator<T>> {
         private static final int UNSET = -1;
 
         private final List<T> testData;
@@ -61,12 +60,12 @@ public class TestDataMatchers {
 
         private String mismatchDescription = null;
 
-        public SplitDataMatcher(List<T> testData) {
+        public SingleSplitDataMatcher(List<T> testData) {
             this.testData = testData;
             this.limit = UNSET;
         }
 
-        public SplitDataMatcher(List<T> testData, int limit) {
+        public SingleSplitDataMatcher(List<T> testData, int limit) {
             if (limit > testData.size()) {
                 throw new IllegalArgumentException(
                         "Limit validation size should be less than number of 
test records");
@@ -85,7 +84,10 @@ public class TestDataMatchers {
             int recordCounter = 0;
             for (T testRecord : testData) {
                 if (!resultIterator.hasNext()) {
-                    mismatchDescription = "Result data is less than test data";
+                    mismatchDescription =
+                            String.format(
+                                    "Expected to have %d records in result, 
but only received %d records",
+                                    limit == UNSET ? testData.size() : limit, 
recordCounter);
                     return false;
                 }
                 T resultRecord = resultIterator.next();
@@ -102,7 +104,11 @@ public class TestDataMatchers {
                 }
             }
             if (limit == UNSET && resultIterator.hasNext()) {
-                mismatchDescription = "Result data is more than test data";
+                mismatchDescription =
+                        String.format(
+                                "Expected to have exactly %d records in 
result, "
+                                        + "but result iterator hasn't reached 
the end",
+                                testData.size());
                 return false;
             } else {
                 return true;
@@ -131,13 +137,13 @@ public class TestDataMatchers {
      */
     public static class MultipleSplitDataMatcher<T> extends 
TypeSafeDiagnosingMatcher<Iterator<T>> {
 
-        private final List<IteratorWithCurrent<T>> testDataIterators = new 
ArrayList<>();
+        List<TestRecords<T>> testRecordsLists = new ArrayList<>();
 
         private String mismatchDescription = null;
 
-        public MultipleSplitDataMatcher(List<List<T>> testRecordsLists) {
-            for (List<T> testRecords : testRecordsLists) {
-                testDataIterators.add(new 
IteratorWithCurrent<>(testRecords.iterator()));
+        public MultipleSplitDataMatcher(List<List<T>> testData) {
+            for (List<T> testRecordsList : testData) {
+                this.testRecordsLists.add(new TestRecords<>(testRecordsList));
             }
         }
 
@@ -147,15 +153,32 @@ public class TestDataMatchers {
                 description.appendText(mismatchDescription);
                 return false;
             }
+
+            int recordCounter = 0;
             while (resultIterator.hasNext()) {
                 final T record = resultIterator.next();
                 if (!matchThenNext(record)) {
-                    mismatchDescription = String.format("Unexpected record 
'%s'", record);
+                    this.mismatchDescription =
+                            generateMismatchDescription(
+                                    String.format(
+                                            "Unexpected record '%s' at 
position %d",
+                                            record, recordCounter),
+                                    resultIterator);
                     return false;
                 }
+                recordCounter++;
             }
+
             if (!hasReachedEnd()) {
-                mismatchDescription = "Result data is less than test data";
+                this.mismatchDescription =
+                        generateMismatchDescription(
+                                String.format(
+                                        "Expected to have exactly %d records 
in result, but only received %d records",
+                                        testRecordsLists.stream()
+                                                .mapToInt(list -> 
list.records.size())
+                                                .sum(),
+                                        recordCounter),
+                                resultIterator);
                 return false;
             } else {
                 return true;
@@ -176,9 +199,12 @@ public class TestDataMatchers {
          * @param record Record from stream
          */
         private boolean matchThenNext(T record) {
-            for (IteratorWithCurrent<T> testDataIterator : testDataIterators) {
-                if (record.equals(testDataIterator.current())) {
-                    testDataIterator.next();
+            for (TestRecords<T> testRecordsList : testRecordsLists) {
+                if (!testRecordsList.hasNext()) {
+                    continue;
+                }
+                if (record.equals(testRecordsList.current())) {
+                    testRecordsList.forward();
                     return true;
                 }
             }
@@ -191,55 +217,67 @@ public class TestDataMatchers {
          * @return True if all pointers have reached the end.
          */
         private boolean hasReachedEnd() {
-            for (IteratorWithCurrent<T> testDataIterator : testDataIterators) {
-                if (testDataIterator.hasNext()) {
+            for (TestRecords<T> testRecordsList : testRecordsLists) {
+                if (testRecordsList.hasNext()) {
                     return false;
                 }
             }
             return true;
         }
-    }
-
-    /**
-     * An iterator wrapper which can access the element that the iterator is 
currently pointing to.
-     *
-     * @param <E> The type of elements returned by this iterator.
-     */
-    static class IteratorWithCurrent<E> implements Iterator<E> {
-
-        private final Iterator<E> originalIterator;
-        private E current;
 
-        public IteratorWithCurrent(Iterator<E> originalIterator) {
-            this.originalIterator = originalIterator;
-            try {
-                current = originalIterator.next();
-            } catch (NoSuchElementException e) {
-                current = null;
+        String generateMismatchDescription(String reason, Iterator<T> 
resultIterator) {
+            final StringBuilder sb = new StringBuilder();
+            sb.append(reason).append("\n");
+            sb.append("Current progress of multiple split test data 
validation:\n");
+            int splitCounter = 0;
+            for (TestRecords<T> testRecordsList : testRecordsLists) {
+                sb.append(
+                        String.format(
+                                "Split %d (%d/%d): \n",
+                                splitCounter++,
+                                testRecordsList.offset,
+                                testRecordsList.records.size()));
+                for (int recordIndex = 0;
+                        recordIndex < testRecordsList.records.size();
+                        recordIndex++) {
+                    sb.append(testRecordsList.records.get(recordIndex));
+                    if (recordIndex == testRecordsList.offset) {
+                        sb.append("\t<----");
+                    }
+                    sb.append("\n");
+                }
+            }
+            if (resultIterator.hasNext()) {
+                sb.append("Remaining received elements after the unexpected 
one: \n");
+                while (resultIterator.hasNext()) {
+                    sb.append(resultIterator.next()).append("\n");
+                }
             }
+            return sb.toString();
         }
 
-        @Override
-        public boolean hasNext() {
-            return current != null;
-        }
+        private static class TestRecords<T> {
+            private int offset = 0;
+            private final List<T> records;
 
-        @Override
-        public E next() {
-            if (current == null) {
-                throw new NoSuchElementException();
+            public TestRecords(List<T> records) {
+                this.records = records;
             }
-            E previous = current;
-            if (originalIterator.hasNext()) {
-                current = originalIterator.next();
-            } else {
-                current = null;
+
+            public T current() {
+                if (!hasNext()) {
+                    return null;
+                }
+                return records.get(offset);
             }
-            return previous;
-        }
 
-        public E current() {
-            return current;
+            public void forward() {
+                ++offset;
+            }
+
+            public boolean hasNext() {
+                return offset < records.size();
+            }
         }
     }
 }
diff --git 
a/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/IteratorWithCurrentTest.java
 
b/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/IteratorWithCurrentTest.java
deleted file mode 100644
index 97a7b48..0000000
--- 
a/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/IteratorWithCurrentTest.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * 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.flink.connectors.test.common.utils;
-
-import 
org.apache.flink.connectors.test.common.utils.TestDataMatchers.IteratorWithCurrent;
-
-import org.junit.jupiter.api.Test;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-
-/** Unit test for {@link IteratorWithCurrent}. */
-public class IteratorWithCurrentTest {
-
-    @Test
-    public void testIterator() {
-        List<Integer> numberList = new ArrayList<>();
-        for (int i = 0; i < 10; i++) {
-            numberList.add(i);
-        }
-        IteratorWithCurrent<Integer> iterator = new 
IteratorWithCurrent<>(numberList.iterator());
-        Integer num = 0;
-        while (iterator.hasNext()) {
-            assertEquals(num, iterator.next());
-            num++;
-        }
-        assertEquals(10, num.intValue());
-        assertNull(iterator.current());
-    }
-
-    @Test
-    public void testEmptyList() {
-        IteratorWithCurrent<Integer> iterator =
-                new IteratorWithCurrent<>(Collections.emptyIterator());
-        assertNull(iterator.current());
-        assertThrows(NoSuchElementException.class, iterator::next);
-    }
-
-    @Test
-    public void testCurrentElement() {
-        List<Integer> numberList = new ArrayList<>();
-        for (int i = 0; i < 10; i++) {
-            numberList.add(i);
-        }
-        IteratorWithCurrent<Integer> iterator = new 
IteratorWithCurrent<>(numberList.iterator());
-        Integer num = 0;
-        while (iterator.hasNext()) {
-            assertEquals(num, iterator.current());
-            assertEquals(num, iterator.next());
-            num++;
-        }
-        assertEquals(10, num.intValue());
-        assertNull(iterator.current());
-    }
-}
diff --git 
a/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/TestDataMatchersTest.java
 
b/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/TestDataMatchersTest.java
index 4e4d8a1..9df0e86 100644
--- 
a/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/TestDataMatchersTest.java
+++ 
b/flink-test-utils-parent/flink-connector-testing/src/test/java/org/apache/flink/connectors/test/common/utils/TestDataMatchersTest.java
@@ -41,7 +41,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
 /** Unit test for {@link TestDataMatchers}. */
 public class TestDataMatchersTest {
     @Nested
-    class SplitDataMatcherTest {
+    class SingleSplitDataMatcherTest {
         private final List<String> testData = Arrays.asList("alpha", "beta", 
"gamma");
 
         @Test
@@ -61,7 +61,7 @@ public class TestDataMatchersTest {
             resultData.set(1, "delta");
             final Iterator<String> resultIterator = resultData.iterator();
 
-            final TestDataMatchers.SplitDataMatcher<String> matcher =
+            final TestDataMatchers.SingleSplitDataMatcher<String> matcher =
                     TestDataMatchers.matchesSplitTestData(testData);
 
             assertMatcherFailedWithDescription(
@@ -76,11 +76,14 @@ public class TestDataMatchersTest {
             resultData.add("delta");
             final Iterator<String> resultIterator = resultData.iterator();
 
-            final TestDataMatchers.SplitDataMatcher<String> matcher =
+            final TestDataMatchers.SingleSplitDataMatcher<String> matcher =
                     TestDataMatchers.matchesSplitTestData(testData);
 
             assertMatcherFailedWithDescription(
-                    resultIterator, matcher, "Result data is more than test 
data");
+                    resultIterator,
+                    matcher,
+                    "Expected to have exactly 3 records in result, "
+                            + "but result iterator hasn't reached the end");
         }
 
         @Test
@@ -89,11 +92,13 @@ public class TestDataMatchersTest {
             resultData.remove(testData.size() - 1);
             final Iterator<String> resultIterator = resultData.iterator();
 
-            final TestDataMatchers.SplitDataMatcher<String> matcher =
+            final TestDataMatchers.SingleSplitDataMatcher<String> matcher =
                     TestDataMatchers.matchesSplitTestData(testData);
 
             assertMatcherFailedWithDescription(
-                    resultIterator, matcher, "Result data is less than test 
data");
+                    resultIterator,
+                    matcher,
+                    "Expected to have 3 records in result, but only received 2 
records");
         }
     }
 
@@ -120,7 +125,22 @@ public class TestDataMatchersTest {
             final TestDataMatchers.MultipleSplitDataMatcher<String> matcher =
                     
TestDataMatchers.matchesMultipleSplitTestData(testDataCollection);
             assertMatcherFailedWithDescription(
-                    result.iterator(), matcher, "Result data is less than test 
data");
+                    result.iterator(),
+                    matcher,
+                    "Expected to have exactly 9 records in result, but only 
received 8 records\n"
+                            + "Current progress of multiple split test data 
validation:\n"
+                            + "Split 0 (2/3): \n"
+                            + "alpha\n"
+                            + "beta\n"
+                            + "gamma\t<----\n"
+                            + "Split 1 (3/3): \n"
+                            + "one\n"
+                            + "two\n"
+                            + "three\n"
+                            + "Split 2 (3/3): \n"
+                            + "1\n"
+                            + "2\n"
+                            + "3\n");
         }
 
         @Test
@@ -130,7 +150,22 @@ public class TestDataMatchersTest {
             final TestDataMatchers.MultipleSplitDataMatcher<String> matcher =
                     
TestDataMatchers.matchesMultipleSplitTestData(testDataCollection);
             assertMatcherFailedWithDescription(
-                    result.iterator(), matcher, "Unexpected record 'delta'");
+                    result.iterator(),
+                    matcher,
+                    "Unexpected record 'delta' at position 9\n"
+                            + "Current progress of multiple split test data 
validation:\n"
+                            + "Split 0 (3/3): \n"
+                            + "alpha\n"
+                            + "beta\n"
+                            + "gamma\n"
+                            + "Split 1 (3/3): \n"
+                            + "one\n"
+                            + "two\n"
+                            + "three\n"
+                            + "Split 2 (3/3): \n"
+                            + "1\n"
+                            + "2\n"
+                            + "3\n");
         }
 
         @Test
@@ -140,7 +175,25 @@ public class TestDataMatchersTest {
             final List<String> result = unionLists(splitA, splitB, reverted);
             final TestDataMatchers.MultipleSplitDataMatcher<String> matcher =
                     
TestDataMatchers.matchesMultipleSplitTestData(testDataCollection);
-            assertMatcherFailedWithDescription(result.iterator(), matcher, 
"Unexpected record '3'");
+            String expectedDescription =
+                    "Unexpected record '3' at position 6\n"
+                            + "Current progress of multiple split test data 
validation:\n"
+                            + "Split 0 (3/3): \n"
+                            + "alpha\n"
+                            + "beta\n"
+                            + "gamma\n"
+                            + "Split 1 (3/3): \n"
+                            + "one\n"
+                            + "two\n"
+                            + "three\n"
+                            + "Split 2 (0/3): \n"
+                            + "1\t<----\n"
+                            + "2\n"
+                            + "3\n"
+                            + "Remaining received elements after the 
unexpected one: \n"
+                            + "2\n"
+                            + "1\n";
+            assertMatcherFailedWithDescription(result.iterator(), matcher, 
expectedDescription);
         }
     }
 

Reply via email to