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

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

                Author: ASF GitHub Bot
            Created on: 31/May/19 22:09
            Start Date: 31/May/19 22:09
    Worklog Time Spent: 10m 
      Work Description: iemejia commented on pull request #8718: [BEAM-7450] - 
Add an unbounded HcatalogIO reader using splittable pardo
URL: https://github.com/apache/beam/pull/8718#discussion_r289017312
 
 

 ##########
 File path: 
sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOTest.java
 ##########
 @@ -163,6 +183,100 @@ public void processElement(ProcessContext c) {
     readAfterWritePipeline.run();
   }
 
+  private Map<String, String> getPartitioned() {
+    String loadDate = "2019-05-14T23:28:04.425Z";
+    int productType = 1;
+    Map<String, String> partitions = new HashMap<>();
+    partitions.put("load_date", String.valueOf(loadDate));
+    partitions.put("product_type", String.valueOf(productType));
+    return partitions;
+  }
+
+  /** Perform end-to-end test of Write-then-Read operation. */
+  @Test
+  @NeedsEmptyTestTablesForUnboundedReads
+  public void testWriteThenUnboundedReadSuccess() throws Exception {
+
+    defaultPipeline
+        .apply(Create.of(buildHCatRecords(TEST_RECORDS_COUNT)))
+        .apply(
+            HCatalogIO.write()
+                
.withConfigProperties(getConfigPropertiesAsMap(service.getHiveConf()))
+                .withDatabase(TEST_DATABASE)
+                .withTable(TEST_TABLE)
+                .withPartition(getPartitioned())
+                .withBatchSize(512L));
+    defaultPipeline.run();
+
+    final HCatalogUnboundedReader reader = HCatalogUnboundedReader.of();
+    reader.setConfig(getConfigPropertiesAsMap(service.getHiveConf()));
+    reader.setDatabase(TEST_DATABASE);
+    reader.setTable(TEST_TABLE);
+    reader.setShouldTreatSourceAsBounded(true);
+
+    final PCollection<HCatRecord> records =
+        readAfterWritePipeline
+            .apply(Create.of(buildReadRequest()))
+            .apply(reader)
+            .setCoder((Coder) WritableCoder.of(DefaultHCatRecord.class));
+
+    final PCollection<String> output =
+        records.apply(
+            ParDo.of(
+                new DoFn<HCatRecord, String>() {
+                  @ProcessElement
+                  public void processElement(ProcessContext c) {
+                    c.output(c.element().get(0).toString());
+                  }
+                }));
+
+    
PAssert.that(output).containsInAnyOrder(getExpectedRecords(TEST_RECORDS_COUNT));
+    readAfterWritePipeline.run();
+  }
+
+  private static List<PartitionPoller.ReadRequest> buildReadRequest() {
+    List<PartitionPoller.ReadRequest> expected = new ArrayList<>();
+    final HCatalogIO.Read read =
+        HCatalogIO.read()
+            .withDatabase(TEST_DATABASE)
+            .withTable(TEST_TABLE)
+            
.withConfigProperties(getConfigPropertiesAsMap(service.getHiveConf()));
+
+    final ImmutableList<String> partitions = ImmutableList.of("load_date", 
"product_type");
+
+    final PartitionPoller.ReadRequest readRequest =
+        new PartitionPoller.ReadRequest(
+            Duration.millis(15000),
+            read,
+            new PartitionComparator(),
+            new WatermarkTimestampConverter(),
+            partitions,
+            "load_date");
+    expected.add(readRequest);
+    return expected;
+  }
+
+  private static class PartitionComparator implements 
SerializableComparator<Partition> {
 
 Review comment:
   `PartitionCreateTimeComparator` maybe.
 
----------------------------------------------------------------
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: 252089)
    Time Spent: 1h 40m  (was: 1.5h)

> Unbounded HCatalogIO Reader using splittable pardos
> ---------------------------------------------------
>
>                 Key: BEAM-7450
>                 URL: https://issues.apache.org/jira/browse/BEAM-7450
>             Project: Beam
>          Issue Type: New Feature
>          Components: sdk-java-core
>            Reporter: Ankit Jhalaria
>            Assignee: Ankit Jhalaria
>            Priority: Minor
>          Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> # Current version of HcatalogIO is a bounded source.
>  # While migrating our jobs to aws, we realized that it would be helpful to 
> have an unbounded hcat reader that can behave as an unbounded source and 
> polls for new partitions as and when they become available.
>  # I have used splittable pardo(s) to do this. There is a flag that can be 
> set to treat this as a bounded source which will terminate if that flag is 
> set.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to