[ 
https://issues.apache.org/jira/browse/SAMZA-1371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16144367#comment-16144367
 ] 

Hao Song commented on SAMZA-1371:
---------------------------------

Below is the stack dump:
"main" #1 prio=5 os_prio=0 tid=0x00007f3ff4026800 nid=0x15b5 waiting on 
condition [0x00007f3ffbd56000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000bdcf65d8> (a 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
        at 
java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at 
java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)
        at 
java.util.concurrent.LinkedBlockingQueue.poll(LinkedBlockingQueue.java:467)
        at 
org.apache.samza.util.BlockingEnvelopeMap.poll(BlockingEnvelopeMap.java:140)
        at 
org.apache.samza.system.SystemStreamPartitionIterator.refresh(SystemStreamPartitionIterator.java:87)
        at 
org.apache.samza.system.SystemStreamPartitionIterator.hasNext(SystemStreamPartitionIterator.java:58)
        at 
scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:41)
        at scala.collection.Iterator$class.foreach(Iterator.scala:727)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
        at 
org.apache.samza.storage.kv.KeyValueStorageEngine.restore(KeyValueStorageEngine.scala:104)
        at 
org.apache.samza.storage.TaskStorageManager$$anonfun$restoreStores$3.apply(TaskStorageManager.scala:187)
        at 
org.apache.samza.storage.TaskStorageManager$$anonfun$restoreStores$3.apply(TaskStorageManager.scala:181)
        at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
        at scala.collection.immutable.Map$Map3.foreach(Map.scala:154)
        at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
        at 
org.apache.samza.storage.TaskStorageManager.restoreStores(TaskStorageManager.scala:181)
        at 
org.apache.samza.storage.TaskStorageManager.init(TaskStorageManager.scala:76)
        at 
org.apache.samza.container.TaskInstance.startStores(TaskInstance.scala:107)
        at 
org.apache.samza.container.SamzaContainer$$anonfun$startStores$2.apply(SamzaContainer.scala:830)
        at 
org.apache.samza.container.SamzaContainer$$anonfun$startStores$2.apply(SamzaContainer.scala:828)
        at scala.collection.Iterator$class.foreach(Iterator.scala:727)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
        at 
scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
        at 
org.apache.samza.container.SamzaContainer.startStores(SamzaContainer.scala:828)
        at 
org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:719)
        at 
org.apache.samza.container.SamzaContainer$.safeMain(SamzaContainer.scala:122)
        at 
org.apache.samza.container.SamzaContainer$.main(SamzaContainer.scala:89)
        at org.apache.samza.container.SamzaContainer.main(SamzaContainer.scala)

   Locked ownable synchronizers:
        - None
=============================
Which points to this restoreStores function TaskStorageManager. Specifically, 
it got stuck at line: 
https://github.com/apache/samza/blob/0.12.0/samza-core/src/main/scala/org/apache/samza/storage/TaskStorageManager.scala#L187
I wish there are more logs but it never reach the next function stopConsumers() 
on line 192 which has another debug statement.

> Some Samza Containers get stuck at "Starting BrokerProxy for 
> hostname:portnum" while others seem to be fine
> -----------------------------------------------------------------------------------------------------------
>
>                 Key: SAMZA-1371
>                 URL: https://issues.apache.org/jira/browse/SAMZA-1371
>             Project: Samza
>          Issue Type: Bug
>          Components: container
>    Affects Versions: 0.11.0
>         Environment: Samza version: 0.11
> Kafka version: 0.11.0.0
>            Reporter: Ak Ka
>            Priority: Blocker
>
> We have multiple Samza apps using local store that have this issue. Some 
> containers get stuck on "Starting BrokerProxy for hostname:portnum" while 
> others seem to work as expected.  
> Here is the log:
> stuck:
> ```
> [...]
> 2017-07-25 17:11:26.546 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Creating new SimpleConsumer for host hostname:portnum for system kafka
> 2017-07-25 17:11:26.547 [main] org.apache.samza.system.kafka.GetOffset [INFO] 
> Validating offset 0 for topic and partition 
> [prod.localStateChangeLog.prod.AlertsOrganizerInstant_matcherValidation,2]
> 2017-07-25 17:11:26.648 [main] org.apache.samza.system.kafka.GetOffset [INFO] 
> Able to successfully read from offset 0 for topic and partition 
> [prod.localStateChangeLog.prod.AlertsOrganizerInstant_matcherValidation,2]. 
> Using it to instantiate consumer.
> 2017-07-25 17:11:26.649 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Starting BrokerProxy for hostname:portnum
> // it's dead, Jim
> ```
> healthy:
> ```
> [...]
> 2017-07-25 17:11:26.920 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Creating new SimpleConsumer for host hostname:portnum for system kafka
> 2017-07-25 17:11:26.921 [main] org.apache.samza.system.kafka.GetOffset [INFO] 
> Validating offset 0 for topic and partition 
> [prod.localStateChangeLog.prod.AlertsOrganizerInstant_matcherValidation,1]
> 2017-07-25 17:11:27.023 [main] org.apache.samza.system.kafka.GetOffset [INFO] 
> Able to successfully read from offset 0 for topic and partition 
> [prod.localStateChangeLog.prod.AlertsOrganizerInstant_matcherValidation,1]. 
> Using it to instantiate consumer.
> 2017-07-25 17:11:27.023 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Starting BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.194 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Shutting down BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.194 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] closing simple consumer...
> 2017-07-25 17:11:29.239 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.DefaultFetchSimpleConsumer [INFO] Reconnect due 
> to socket error: java.nio.channels.ClosedChannelException
> 2017-07-25 17:11:29.244 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [WARN] Restarting consumer due to 
> java.nio.channels.ClosedChannelException. Releasing ownership of all 
> partitions, and restarting consumer. Turn on debugging to get a full stack 
> trace.
> 2017-07-25 17:11:29.247 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.KafkaSystemConsumer [INFO] Abdicating for 
> [prod.localStateChangeLog.prod.AlertsOrganizerInstant_alertSetting,1]
> 2017-07-25 17:11:29.247 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.KafkaSystemConsumer [INFO] Refreshing brokers 
> for: 
> Map([prod.localStateChangeLog.prod.AlertsOrganizerInstant_alertSetting,1] -> 
> 13572)
> 2017-07-25 17:11:29.247 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [INFO] Shutting down due to 
> interrupt.
> 2017-07-25 17:11:29.247 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Shutting down BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.248 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] closing simple consumer...
> 2017-07-25 17:11:29.265 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [INFO] Shutting down due to 
> interrupt.
> 2017-07-25 17:11:29.265 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Shutting down BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.265 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] closing simple consumer...
> 2017-07-25 17:11:29.523 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [INFO] Shutting down due to 
> interrupt.
> 2017-07-25 17:11:29.524 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Shutting down BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.524 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] closing simple consumer...
> 2017-07-25 17:11:29.601 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [INFO] Shutting down due to 
> interrupt.
> 2017-07-25 17:11:29.602 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] Shutting down BrokerProxy for hostname:portnum
> 2017-07-25 17:11:29.602 [main] org.apache.samza.system.kafka.BrokerProxy 
> [INFO] closing simple consumer...
> 2017-07-25 17:11:29.663 [SAMZA-BROKER-PROXY-BrokerProxy thread pointed at 
> hostname:portnum for client samza_consumer-prod_AlertsOrganizerInstant-1] 
> org.apache.samza.system.kafka.BrokerProxy [INFO] Shutting down due to 
> interrupt.
> 2017-07-25 17:11:29.668 [main] org.apache.samza.container.SamzaContainer 
> [INFO] Starting host statistics monitor
> 2017-07-25 17:11:29.670 [main] org.apache.samza.container.SamzaContainer 
> [INFO] Registering task instances with producers.
> 2017-07-25 17:11:29.674 [main] org.apache.samza.container.SamzaContainer 
> [INFO] Starting producer multiplexer.
> 2017-07-25 17:11:29.675 [main] org.apache.samza.container.SamzaContainer 
> [INFO] Initializing stream tasks.
> 2017-07-25 17:11:29.676 [main] 
> com.company.samza.app.companyStreamingAppWrapper [INFO] Initializing instance 
> of streaming application
> 2017-07-25 17:11:29.681 [main] 
> com.company.samza.app.companyStreamingAppWrapper [INFO] First initialization. 
> Setting up Guice container with configuration 
> companyStreamingAppWrapperConfiguration{company.app.name=AlertsOrganizerInstant,
>  company.appgroup=aws, company.env=prod, 
> company.guice.module=com.company.notifications.Alerts.organizer..AlertsOrganizerModule}
> 2017-07-25 17:11:30.118 [main] com.company.config.guice.configModule [INFO] 
> configModule loaded requested override file 
> '/storage/data/secure/config/AnalyticsServiceClient.cfg'
> 2017-07-25 17:11:30.480 [main] 
> com.company.samza.dataService.SamzaSessionFactoriesModule [INFO] Loading prod 
> dbConfig from /data/config/prod.database.properties
> // Hibernate stuff (i.e. our code is hit)
> ```



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to