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

ASF GitHub Bot logged work on GOBBLIN-1040:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 02/Mar/20 02:42
            Start Date: 02/Mar/20 02:42
    Worklog Time Spent: 10m 
      Work Description: sv2000 commented on pull request #2900: [GOBBLIN-1040] 
HighLevelConsumer re-design by removing references to …
URL: https://github.com/apache/incubator-gobblin/pull/2900#discussion_r386170305
 
 

 ##########
 File path: 
gobblin-runtime/src/main/java/org/apache/gobblin/runtime/kafka/HighLevelConsumer.java
 ##########
 @@ -83,14 +95,58 @@
   @Getter
   private MetricContext metricContext;
   private Counter messagesRead;
-  private ConsumerConnector consumer;
-  private ExecutorService executor;
+  @Getter
+  private final GobblinKafkaConsumerClient gobblinKafkaConsumerClient;
+  private final ScheduledExecutorService mainExecutor;
+  private final ExecutorService queueExecutor;
+  private ScheduledExecutorService offsetCommitExecutor;
+  private final BlockingQueue[] queues;
+  private final AtomicInteger recordsProcessed;
+  private final Map<KafkaPartition, Long> partitionOffsetsToCommit;
+  private final boolean enableAutoCommit;
+  private final int offsetsCommitNumRecordsThreshold;
+  private final int offsetsCommitTimeThresholdSecs;
+  private long lastCommitTime = System.currentTimeMillis();
+
+  private static final Config FALLBACK =
+      ConfigFactory.parseMap(ImmutableMap.<String, Object>builder()
+          .put(GROUP_ID_KEY, DEFAULT_GROUP_ID)
+          .put(ENABLE_AUTO_COMMIT_KEY, DEFAULT_AUTO_COMMIT_VALUE)
+          .build());
 
   public HighLevelConsumer(String topic, Config config, int numThreads) {
     this.topic = topic;
     this.numThreads = numThreads;
-    this.config = config;
-    this.consumerConfig = createConsumerConfig(config);
+    this.config = config.withFallback(FALLBACK);
+    this.gobblinKafkaConsumerClient = createConsumerClient(config);
+    this.gobblinKafkaConsumerClient.subscribe(this.topic);
+    this.mainExecutor = 
Executors.newSingleThreadScheduledExecutor(ExecutorsUtils.newThreadFactory(Optional.of(log),
 Optional.of("HighLevelConsumerThread")));
+    this.queueExecutor = Executors.newFixedThreadPool(this.numThreads, 
ExecutorsUtils.newThreadFactory(Optional.of(log), 
Optional.of("QueueProcessor-%d")));
+    this.queues = new LinkedBlockingQueue[numThreads];
+    for(int i=0;i<queues.length;i++) {
+      this.queues[i] = new LinkedBlockingQueue();
+    }
+    this.recordsProcessed = new AtomicInteger(0);
+    this.partitionOffsetsToCommit = new ConcurrentHashMap<>();
+    this.enableAutoCommit = ConfigUtils.getBoolean(config, 
ENABLE_AUTO_COMMIT_KEY, DEFAULT_AUTO_COMMIT_VALUE);
+    this.offsetsCommitNumRecordsThreshold = ConfigUtils.getInt(config, 
OFFSET_COMMIT_NUM_RECORDS_THRESHOLD_KEY, 
DEFAULT_OFFSET_COMMIT_NUM_RECORDS_THRESHOLD);
+    this.offsetsCommitTimeThresholdSecs = ConfigUtils.getInt(config, 
OFFSET_COMMIT_TIME_THRESHOLD_SECS_KEY, 
DEFAULT_OFFSET_COMMIT_TIME_THRESHOLD_SECS);
+  }
+
+  protected GobblinKafkaConsumerClient createConsumerClient(Config config) {
+    String kafkaConsumerClientClass = ConfigUtils.getString(config, 
CONSUMER_CLIENT_FACTORY_CLASS_KEY,
+        DEFAULT_CONSUMER_CLIENT_FACTORY_CLASS);
+
+    try {
+      Class clientFactoryClass = Class.forName(kafkaConsumerClientClass);
+      final GobblinKafkaConsumerClient.GobblinKafkaConsumerClientFactory 
factory =
+          (GobblinKafkaConsumerClient.GobblinKafkaConsumerClientFactory)
+              ConstructorUtils.invokeConstructor(clientFactoryClass);
+
+      return factory.create(config);
+    } catch (ClassNotFoundException | NoSuchMethodException | 
IllegalAccessException | InstantiationException | InvocationTargetException e) {
 
 Review comment:
   Can we lump all exceptions into ReflectiveOperationException?
 
----------------------------------------------------------------
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: 395731)
    Time Spent: 2h  (was: 1h 50m)

> Fix High level consumer 
> ------------------------
>
>                 Key: GOBBLIN-1040
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-1040
>             Project: Apache Gobblin
>          Issue Type: Improvement
>            Reporter: Vikram Bohra
>            Priority: Major
>          Time Spent: 2h
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to