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

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

                Author: ASF GitHub Bot
            Created on: 12/Jun/19 15:42
            Start Date: 12/Jun/19 15:42
    Worklog Time Spent: 10m 
      Work Description: jkff commented on pull request #8718: [BEAM-7450] Add 
an unbounded HcatalogIO reader using splittable pardo
URL: https://github.com/apache/beam/pull/8718#discussion_r292985827
 
 

 ##########
 File path: 
sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionPollerFn.java
 ##########
 @@ -0,0 +1,141 @@
+/*
+ * 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.io.hcatalog;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.sdk.io.hcatalog.HCatalogIO.Read;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.UnboundedPerElement;
+import org.apache.beam.sdk.transforms.SerializableComparator;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableList;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Unbounded poller to listen for new partitions. */
+@UnboundedPerElement
+class PartitionPollerFn extends DoFn<Read, Read> {
 
 Review comment:
   Yes, it does mean that we don't need an SDF here at all; and yes, this would 
likely work for KafkaIO too. Watch would poll for the current list of 
partitions, and emit new ones as they appear without re-emitting the already 
known ones, just like when we watch for new files.
   
   It also means that the current PR would not be introducing any new SDFs, but 
instead merely reusing an existing one (Watch) - which is a very good thing and 
in line with SDF's position as enabling more composable/reusable IO APIs as 
opposed to people having to reimplement similar features in different IOs.
 
----------------------------------------------------------------
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:
us...@infra.apache.org


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

    Worklog Id:     (was: 258841)
    Time Spent: 5h  (was: 4h 50m)

> 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: 5h
>  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