Github user rmetzger commented on a diff in the pull request:

    https://github.com/apache/bahir-flink/pull/8#discussion_r89286046
  
    --- Diff: 
flink-connector-akka/src/main/java/org/apache/flink/streaming/connectors/akka/AkkaSource.java
 ---
    @@ -0,0 +1,174 @@
    +/*
    + * 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.flink.streaming.connectors.akka;
    +
    +import akka.actor.ActorRef;
    +import akka.actor.ActorSystem;
    +import akka.actor.PoisonPill;
    +import akka.actor.Props;
    +import com.typesafe.config.Config;
    +import com.typesafe.config.ConfigFactory;
    +import org.apache.flink.api.common.functions.StoppableFunction;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.connectors.akka.utils.ReceiverActor;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Map;
    +import java.util.Properties;
    +
    +/**
    + * Implementation of {@link SourceFunction} specialized to read messages
    + * from Akka actors.
    + */
    +public class AkkaSource extends RichSourceFunction<Object>
    +  implements StoppableFunction {
    +
    +  private static final Logger LOG = 
LoggerFactory.getLogger(AkkaSource.class);
    +
    +  private static final long serialVersionUID = 1L;
    +
    +  // --- Fields set by the constructor
    +
    +  private final Class<?> classForActor;
    +
    +  private final String actorName;
    +
    +  private final String urlOfPublisher;
    +
    +  // --- Runtime fields
    +  private transient ActorSystem receiverActorSystem;
    +  private transient ActorRef receiverActor;
    +  private transient Object waitLock;
    +  private transient boolean running = true;
    +
    +  protected transient boolean autoAck;
    +
    +  /**
    +   * Creates {@link AkkaSource} for Streaming
    +   *
    +   * @param actorName Receiver Actor name
    +   * @param urlOfPublisher tcp url of the publisher or feeder actor
    +   */
    +  public AkkaSource(String actorName,
    +          String urlOfPublisher) {
    +    super();
    +    this.classForActor = ReceiverActor.class;
    +    this.actorName = actorName;
    +    this.urlOfPublisher = urlOfPublisher;
    +  }
    +
    +  @Override
    +  public void open(Configuration parameters) throws Exception {
    +    Properties customProperties = new Properties();
    +    parameters.addAllToProperties(customProperties);
    +
    +    waitLock = new Object();
    +    receiverActorSystem = createDefaultActorSystem(customProperties);
    +
    +    if (customProperties.containsKey("akka.remote.auto-ack") &&
    +      customProperties.getProperty("akka.remote.auto-ack").equals("on")) {
    +      autoAck = true;
    +    } else {
    +      autoAck = false;
    +    }
    +  }
    +
    +  @Override
    +  public void run(SourceFunction.SourceContext<Object> ctx) throws 
Exception {
    +    LOG.info("Starting the Receiver actor {}", actorName);
    +    receiverActor = receiverActorSystem.actorOf(
    +      Props.create(classForActor, ctx, urlOfPublisher, autoAck), 
actorName);
    +
    +    running = true;
    +    LOG.info("Started the Receiver actor {} successfully", actorName);
    +    receiverActorSystem.awaitTermination();
    +
    +    while (running) {
    +      synchronized (waitLock) {
    +        waitLock.wait(100L);
    --- End diff --
    
    I don't think the wait lock is needed anymore when doing 
`awaitTermination()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to