TheNeuralBit commented on a change in pull request #12827:
URL: https://github.com/apache/beam/pull/12827#discussion_r499002114



##########
File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.beam.sdk.extensions.sql.meta.provider.kafka;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Test for BeamKafkaCSVTable. */

Review comment:
       ```suggestion
   /** Test utility for BeamKafkaTable implementations. */
   ```

##########
File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableAvroTest.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.beam.sdk.extensions.sql.meta.provider.kafka;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.List;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.utils.AvroUtils;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.sdk.values.TypeDescriptor;
+import org.apache.beam.sdk.values.TypeDescriptors;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+public class BeamKafkaTableAvroTest extends BeamKafkaTableTest {
+  private static final Schema EMPTY_SCHEMA = Schema.builder().build();
+
+  private final Schema SCHEMA =

Review comment:
       I think I've seem issues with SchemaCoder and static members in the 
past, is that why this isn't static?

##########
File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.beam.sdk.extensions.sql.meta.provider.kafka;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Test for BeamKafkaCSVTable. */
+public abstract class BeamKafkaTableTest {
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+
+  protected static final List<String> TOPICS = ImmutableList.of("topic1", 
"topic2");
+
+  private static final Map<String, BeamSqlTable> tables = new HashMap<>();
+
+  protected static BeamSqlEnv env = BeamSqlEnv.readOnly("test", tables);
+
+  protected abstract KafkaTestRecord<?> createKafkaTestRecord(String key, int 
i, long timestamp);
+
+  protected abstract KafkaTestTable getTestTable(int numberOfPartitions);
+
+  protected abstract BeamKafkaTable getBeamKafkaTable();
+
+  protected abstract PCollection<KV<byte[], byte[]>> 
applyRowToBytesKV(PCollection<Row> rows);
+
+  protected abstract List<Object> listFrom(int i);
+
+  protected abstract Schema getSchema();

Review comment:
       Please add a brief description of the abstract methods so it's clear for 
future implementers what they should do

##########
File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.beam.sdk.extensions.sql.meta.provider.kafka;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+
+/** Test for BeamKafkaCSVTable. */
+public abstract class BeamKafkaTableTest {
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+
+  protected static final List<String> TOPICS = ImmutableList.of("topic1", 
"topic2");
+
+  private static final Map<String, BeamSqlTable> tables = new HashMap<>();
+
+  protected static BeamSqlEnv env = BeamSqlEnv.readOnly("test", tables);
+
+  protected abstract KafkaTestRecord<?> createKafkaTestRecord(String key, int 
i, long timestamp);
+
+  protected abstract KafkaTestTable getTestTable(int numberOfPartitions);
+
+  protected abstract BeamKafkaTable getBeamKafkaTable();
+
+  protected abstract PCollection<KV<byte[], byte[]>> 
applyRowToBytesKV(PCollection<Row> rows);
+
+  protected abstract List<Object> listFrom(int i);
+
+  protected abstract Schema getSchema();
+
+  @Test
+  public void testOrderedArrivalSinglePartitionRate() {
+    KafkaTestTable table = getTestTable(1);
+    for (int i = 0; i < 100; i++) {
+      table.addRecord(createKafkaTestRecord("k" + i, i, 500L * i));
+    }
+
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertEquals(2d, stats.getRate(), 0.001);
+  }
+
+  @Test
+  public void testOrderedArrivalMultiplePartitionsRate() {
+    KafkaTestTable table = getTestTable(3);
+    for (int i = 0; i < 100; i++) {
+      table.addRecord(createKafkaTestRecord("k" + i, i, 500L * i));
+    }
+
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertEquals(2d, stats.getRate(), 0.001);
+  }
+
+  @Test
+  public void testOnePartitionAheadRate() {
+    KafkaTestTable table = getTestTable(3);
+    for (int i = 0; i < 100; i++) {
+      table.addRecord(createKafkaTestRecord("1", i, 1000L * i));
+      table.addRecord(createKafkaTestRecord("2", i, 500L * i));
+    }
+
+    table.setNumberOfRecordsForRate(20);
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertEquals(1d, stats.getRate(), 0.001);
+  }
+
+  @Test
+  public void testLateRecords() {
+    KafkaTestTable table = getTestTable(3);
+
+    table.addRecord(createKafkaTestRecord("1", 132, 1000L));
+    for (int i = 0; i < 98; i++) {
+      table.addRecord(createKafkaTestRecord("1", i, 500L));
+    }
+    table.addRecord(createKafkaTestRecord("1", 133, 2000L));
+
+    table.setNumberOfRecordsForRate(200);
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertEquals(1d, stats.getRate(), 0.001);
+  }
+
+  @Test
+  public void testAllLate() {
+    KafkaTestTable table = getTestTable(3);
+
+    table.addRecord(createKafkaTestRecord("1", 132, 1000L));
+    for (int i = 0; i < 98; i++) {
+      table.addRecord(createKafkaTestRecord("1", i, 500L));
+    }
+
+    table.setNumberOfRecordsForRate(200);
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertTrue(stats.isUnknown());
+  }
+
+  @Test
+  public void testEmptyPartitionsRate() {
+    KafkaTestTable table = getTestTable(3);
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertTrue(stats.isUnknown());
+  }
+
+  @Test
+  public void allTheRecordsSameTimeRate() {
+    KafkaTestTable table = getTestTable(3);
+    for (int i = 0; i < 100; i++) {
+      table.addRecord(createKafkaTestRecord("key" + i, i, 1000L));
+    }
+    BeamTableStatistics stats = table.getTableStatistics(null);
+    Assert.assertTrue(stats.isUnknown());
+  }
+
+  @Test
+  public void testRecorderDecoder() {
+    BeamKafkaTable kafkaTable = getBeamKafkaTable();
+
+    PCollection<Row> initialRows = pipeline.apply(Create.of(generateRow(1), 
generateRow(2)));
+
+    PCollection<KV<byte[], byte[]>> bytesKV = applyRowToBytesKV(initialRows);
+    PCollection<Row> result = 
bytesKV.apply(kafkaTable.getPTransformForInput());
+
+    PAssert.that(result).containsInAnyOrder(generateRow(1), generateRow(2));
+    pipeline.run();
+  }

Review comment:
       I don't think this test is meaningfully different from 
`testRecorderEncoder`. It looks like both of the implementations of 
`applyRowToBytesKV` are effectively the same as 
`.apply(kafkaTable.getPTransformForOutput())`, so we're not really getting a 
good signal that the encoder and decoder work as intended on their own.
   
   It would be better if there were an abstract method like 
`generateEncodedPayload(i)` that returns the encoded counterpart for 
`generateRow(i)`. Crucially, this method shouldn't use any of the code that 
we're testing here (like `beamRow2CsvLines`, or 
`AvroUtils.getRowToAvroBytesFunction(getSchema())`), it should instead generate 
the test data from scratch. This isn't too hard for CSV since its just a simple 
String. It's harder for Avro, but still doable, I think you can make it work by 
creating GenericRecord instances and encoding them.
   
   Then there could be tests like
   - generate input with `generateEncodedPayload`, apply 
`getPTransformForInput()`, verify it matches data created with `generateRow`.
   - generate input with `generateRow`, apply `getPTransformForOutput()`, 
verify it matches data created with `generateEncodedPayload`.
   - we could also have a round-trip test, like what's done in 
`testRecorderEncoder` now
   
   Note `generateEncodedPayload` could also be re-used in 
`createKafkaTestRecord` and it could have a concrete implementation in 
`BeamKafkaTableTest` as `return KafkaTestRecord.create(key, 
generateEncodedPayload(i), "topic1", timestamp);`




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to