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

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

                Author: ASF GitHub Bot
            Created on: 14/Jun/19 17:14
            Start Date: 14/Jun/19 17:14
    Worklog Time Spent: 10m 
      Work Description: jkff commented on pull request #8718: [BEAM-7450] 
Support unbounded reads with HCatalogIO
URL: https://github.com/apache/beam/pull/8718#discussion_r293900552
 
 

 ##########
 File path: 
sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java
 ##########
 @@ -0,0 +1,111 @@
+/*
+ * 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.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 List<String> partitionCols = readRequest.getPartitionCols();
+    checkArgument(
+        values.size() == partitionCols.size(),
+        "Number of input partitions should be equal to the values of list 
partition values.");
+
+    List<String> filter = new ArrayList<>();
+    for (int i = 0; i < partitionCols.size(); i++) {
+      filter.add(partitionCols.get(i) + "=" + "'" + values.get(i) + "'");
+    }
+    final String filterString = String.join(" and ", filter);
+
+    ReadEntity entity =
+        new ReadEntity.Builder()
+            .withDatabase(readRequest.getDatabase())
+            .withFilter(filterString)
+            .withTable(readRequest.getTable())
+            .build();
+    // pass the 'desired' split count as an hint to the API
+    Map<String, String> configProps = new 
HashMap<>(readRequest.getConfigProperties());
+    configProps.put(
+        HCatConstants.HCAT_DESIRED_PARTITION_NUM_SPLITS, 
String.valueOf(desiredSplitCount));
+    return DataTransferFactory.getHCatReader(entity, 
configProps).prepareRead();
+  }
+
+  @ProcessElement
+  public void processElement(ProcessContext c) throws Exception {
+    final Read readRequest = c.element().getKey();
+    final Integer partitionIndexToRead = c.element().getValue();
+    ReaderContext readerContext = getReaderContext(readRequest, 
partitionIndexToRead);
+    for (int i = 0; i < readerContext.numSplits(); i++) {
 
 Review comment:
   Why do we need a nested loop here: is it possible to just read the whole 
partition without "splitting" it?
   
   If you want to read the splits of a partition in parallel, then you need to 
split this into two DoFn's with a reshuffle in between.
 
----------------------------------------------------------------
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: 260579)
    Time Spent: 10h 40m  (was: 10.5h)

> Support unbounded reads with HCatalogIO
> ---------------------------------------
>
>                 Key: BEAM-7450
>                 URL: https://issues.apache.org/jira/browse/BEAM-7450
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-hcatalog
>            Reporter: Ankit Jhalaria
>            Assignee: Ankit Jhalaria
>            Priority: Minor
>             Fix For: 2.14.0
>
>          Time Spent: 10h 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