[ 
https://issues.apache.org/jira/browse/BEAM-7896?focusedWorklogId=291550&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-291550
 ]

ASF GitHub Bot logged work on BEAM-7896:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 08/Aug/19 20:04
            Start Date: 08/Aug/19 20:04
    Worklog Time Spent: 10m 
      Work Description: akedin commented on pull request #9298: [BEAM-7896] 
Implementing RateEstimation for KafkaTable 
URL: https://github.com/apache/beam/pull/9298#discussion_r312218335
 
 

 ##########
 File path: 
sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaCSVTableIT.java
 ##########
 @@ -0,0 +1,296 @@
+/*
+ * 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 static org.apache.beam.sdk.schemas.Schema.FieldType.INT32;
+import static org.apache.beam.sdk.schemas.Schema.toSchema;
+
+import com.alibaba.fastjson.JSON;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import javax.annotation.Nullable;
+import org.apache.beam.runners.direct.DirectOptions;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.state.BagState;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+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.grpc.v1p21p0.com.google.common.base.MoreObjects;
+import 
org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+
+/**
+ * This is an integration test for KafkaCSVTable. There should be a kafka 
server running and the
+ * address should be passed to it.
+ */
+public class KafkaCSVTableIT {
+
+  @Rule public transient TestPipeline pipeline = TestPipeline.create();
+  //  @Rule public String unique_topic = "topic_"+System.currentTimeMillis();
+
+  private static final Schema TEST_TABLE_SCHEMA =
+      Schema.builder()
+          .addNullableField("order_id", Schema.FieldType.INT32)
+          .addNullableField("member_id", Schema.FieldType.INT32)
+          .addNullableField("item_name", Schema.FieldType.INT32)
+          .build();
+
+  @BeforeClass
+  public static void prepare() {
+    PipelineOptionsFactory.register(KafkaOptions.class);
+  }
+
+  @Test
+  @SuppressWarnings("FutureReturnValueIgnored")
+  public void testFake2() throws BeamKafkaTable.NoEstimationException {
+    KafkaOptions kafkaOptions = pipeline.getOptions().as(KafkaOptions.class);
+    Table table =
+        Table.builder()
+            .name("kafka_table")
+            .comment("kafka" + " table")
+            .location("")
+            .schema(
+                Stream.of(
+                        Schema.Field.nullable("order_id", INT32),
+                        Schema.Field.nullable("member_id", INT32),
+                        Schema.Field.nullable("item_name", INT32))
+                    .collect(toSchema()))
+            .type("kafka")
+            
.properties(JSON.parseObject(getKafkaPropertiesString(kafkaOptions)))
+            .build();
+    BeamKafkaTable kafkaTable = (BeamKafkaTable) new 
KafkaTableProvider().buildBeamSqlTable(table);
+    produceSomeRecordsWithDelay(100, 20);
+    double rate1 = kafkaTable.computeRate(20);
+    produceSomeRecordsWithDelay(100, 10);
+    double rate2 = kafkaTable.computeRate(20);
+    Assert.assertTrue(rate2 > rate1);
+  }
+
+  private String getKafkaPropertiesString(KafkaOptions kafkaOptions) {
+    return "{ \"bootstrap.servers\" : \""
+        + kafkaOptions.getKafkaBootstrapServerAddress()
+        + "\",\"topics\":[\""
+        + kafkaOptions.getKafkaTopic()
+        + "\"] }";
+  }
+
+  static final transient Map<Long, Boolean> FLAG = new ConcurrentHashMap<>();
+
+  @Test
+  public void testFake() throws InterruptedException {
+    KafkaOptions kafkaOptions = pipeline.getOptions().as(KafkaOptions.class);
+    pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false);
+    String createTableString =
+        "CREATE EXTERNAL TABLE kafka_table(\n"
+            + "order_id INTEGER, \n"
+            + "member_id INTEGER, \n"
+            + "item_name INTEGER \n"
+            + ") \n"
+            + "TYPE 'kafka' \n"
+            + "LOCATION '"
+            + "'\n"
+            + "TBLPROPERTIES '"
+            + getKafkaPropertiesString(kafkaOptions)
+            + "'";
+    TableProvider tb = new KafkaTableProvider();
+    BeamSqlEnv env = BeamSqlEnv.inMemory(tb);
+
+    env.executeDdl(createTableString);
+
+    PCollection<Row> queryOutput =
+        BeamSqlRelUtils.toPCollection(pipeline, env.parseQuery("SELECT * FROM 
kafka_table"));
+
+    queryOutput
+        .apply(ParDo.of(new FakeKvPair()))
+        .apply(
+            "waitForSuccess",
+            ParDo.of(
+                new StreamAssertEqual(
+                    //                    flag,
+                    ImmutableSet.of(
+                        row(TEST_TABLE_SCHEMA, 0, 1, 0),
+                        row(TEST_TABLE_SCHEMA, 1, 2, 1),
+                        row(TEST_TABLE_SCHEMA, 2, 3, 2)))));
+    queryOutput.apply(logRecords(""));
+    pipeline.run();
+    TimeUnit.MILLISECONDS.sleep(3000);
+    produceSomeRecords(3);
+
+    for (int i = 0; i < 200; i++) {
+      if (FLAG.getOrDefault(pipeline.getOptions().getOptionsId(), false)) {
+        return;
+      }
+      TimeUnit.MILLISECONDS.sleep(60);
+    }
+    Assert.fail();
+  }
+
+  private static MapElements<Row, Void> logRecords(String suffix) {
+    return MapElements.via(
+        new SimpleFunction<Row, Void>() {
+          @Override
+          public @Nullable Void apply(Row input) {
+            System.out.println(input.getValues() + suffix);
+            return null;
+          }
+        });
+  }
+
+  /** This is made because DoFn with states should get KV as input. */
+  public static class FakeKvPair extends DoFn<Row, KV<String, Row>> {
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(KV.of("", c.element()));
 
 Review comment:
   I would explicitly say something like `"fake_key"`, easier to debug
 
----------------------------------------------------------------
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]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 291550)
    Time Spent: 1h 20m  (was: 1h 10m)

> Rate estimation for Kafka Table
> -------------------------------
>
>                 Key: BEAM-7896
>                 URL: https://issues.apache.org/jira/browse/BEAM-7896
>             Project: Beam
>          Issue Type: New Feature
>          Components: dsl-sql
>            Reporter: Alireza Samadianzakaria
>            Assignee: Alireza Samadianzakaria
>            Priority: Major
>          Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> Currently, KafkaTable returns UNKNOWN statistics for its rate. 
> We can use previously arrived tuples to estimate the rate and return correct 
> statistics (See 
> [https://docs.google.com/document/d/1vi1PBBu5IqSy-qZl1Gk-49CcANOpbNs1UAud6LnOaiY|https://docs.google.com/document/d/1vi1PBBu5IqSy-qZl1Gk-49CcANOpbNs1UAud6LnOaiY/])
>  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to