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

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

                Author: ASF GitHub Bot
            Created on: 14/Jun/19 08:24
            Start Date: 14/Jun/19 08:24
    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_r293704688
 
 

 ##########
 File path: 
sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java
 ##########
 @@ -0,0 +1,115 @@
+/*
+ * 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 static 
org.apache.beam.vendor.guava.v20_0.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+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.values.KV;
+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.apache.hive.hcatalog.common.HCatConstants;
+import org.apache.hive.hcatalog.data.HCatRecord;
+import org.apache.hive.hcatalog.data.transfer.DataTransferFactory;
+import org.apache.hive.hcatalog.data.transfer.HCatReader;
+import org.apache.hive.hcatalog.data.transfer.ReadEntity;
+import org.apache.hive.hcatalog.data.transfer.ReaderContext;
+
+/** Reads partition at a given index. */
+class PartitionReaderFn extends DoFn<KV<Read, Integer>, HCatRecord> {
+  private transient IMetaStoreClient metaStoreClient;
+  private Map<String, String> configProperties;
+
+  public PartitionReaderFn(Map<String, String> configProperties) {
+    this.configProperties = configProperties;
+  }
+
+  private ReaderContext getReaderContext(Read readRequest, Integer 
partitionIndexToRead)
+      throws Exception {
+    final List<Partition> partitions =
+        metaStoreClient.listPartitions(
+            readRequest.getDatabase(), readRequest.getTable(), 
Short.MAX_VALUE);
+    final Partition partition = partitions.get(partitionIndexToRead);
+    checkArgument(
+        partition != null, "Unable to find a partition to read at index " + 
partitionIndexToRead);
+
+    final int desiredSplitCount = HCatalogUtils.getSplitCount(readRequest, 
partition);
+    final List<String> values = partition.getValues();
+    final ImmutableList<String> partitionCols = readRequest.getPartitionCols();
 
 Review comment:
   What happens if the user does not define partitionCols as it was before. 
Maybe worth to make it an empty list as default when building the Read.
 
----------------------------------------------------------------
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: 260220)
    Time Spent: 8h 20m  (was: 8h 10m)

> 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: 8h 20m
>  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