Skip to content

Commit

Permalink
Update TestFlinkTableSource to match new version from 1.14
Browse files Browse the repository at this point in the history
  • Loading branch information
kbendick committed Apr 29, 2022
1 parent 5d7cc57 commit 6883bd7
Showing 1 changed file with 13 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ public void testLimitPushDown() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertEquals("Should produce the expected records", expectedList, resultExceed);
assertSameElements(expectedList, resultExceed);

String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME);
List<Row> mixedResult = sql(sqlMixed);
Expand All @@ -136,7 +136,7 @@ public void testNoFilterPushDown() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), result.toArray());
assertSameElements(expectedRecords, result);
Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
}

Expand Down Expand Up @@ -187,7 +187,7 @@ public void testFilterPushDownNoEqual() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertEquals("Should produce the expected record", expectedNE, resultNE);
assertSameElements(expectedNE, resultNE);
Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
}
Expand Down Expand Up @@ -226,7 +226,7 @@ public void testFilterPushDownOr() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expectedOR, resultOr);
assertSameElements(expectedOR, resultOr);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand All @@ -244,7 +244,7 @@ public void testFilterPushDownGreaterThan() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertEquals("Should produce the expected record", expectedGT, resultGT);
assertSameElements(expectedGT, resultGT);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand All @@ -271,7 +271,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expectedGT, resultGT);
assertSameElements(expectedGT, resultGT);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand All @@ -289,7 +289,7 @@ public void testFilterPushDownGreaterThanEqual() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertEquals("Should produce the expected record", expectedGTE, resultGTE);
assertSameElements(expectedGTE, resultGTE);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand All @@ -316,7 +316,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expectedGTE, resultGTE);
assertSameElements(expectedGTE, resultGTE);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand Down Expand Up @@ -403,7 +403,7 @@ public void testFilterPushDownIn() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expectedIN, resultIN);
assertSameElements(expectedIN, resultIN);
Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
}
Expand Down Expand Up @@ -450,7 +450,7 @@ public void testFilterPushDownIsNotNull() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expected, resultNotNull);
assertSameElements(expected, resultNotNull);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
Assert.assertEquals("Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString());
Expand Down Expand Up @@ -493,7 +493,7 @@ public void testFilterPushDownBetween() {
Row.of(1, "iceberg", 10.0),
Row.of(2, "b", 20.0)
);
Assert.assertEquals("Should produce the expected record", expectedBetween, resultBetween);
assertSameElements(expectedBetween, resultBetween);

Assert.assertEquals("Should create only one scan", 1, scanEventCount);
String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)";
Expand Down Expand Up @@ -554,7 +554,7 @@ public void testFilterNotPushDownLike() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertEquals("Should produce the expected record", expectedRecords, resultLike);
assertSameElements(expectedRecords, resultLike);
Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());

sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' ";
Expand All @@ -579,7 +579,7 @@ public void testFilterPushDown2Literal() {
Row.of(2, "b", 20.0),
Row.of(3, null, 30.0)
);
Assert.assertArrayEquals("Should produce the expected record", expectedRecords.toArray(), result.toArray());
assertSameElements(expectedRecords, result);
Assert.assertEquals("Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter());
}

Expand Down

0 comments on commit 6883bd7

Please sign in to comment.