Hey Shekar,

Are your YARN hosts connected to a VPN by chance?

The only time I've seen problems with the localhost was when I ran unit
tests on my Mac while connected to VPN. Apparently, its related to a bug in
Java http://bugs.java.com/view_bug.do?bug_id=7180557

-Jake

On Thu, Aug 4, 2016 at 4:06 PM, Shekar Tippur <ctip...@gmail.com> wrote:

> Is there anyway we can disable this?
>
> - Shekar
>
> On Thu, Aug 4, 2016 at 4:02 PM, Shekar Tippur <ctip...@gmail.com> wrote:
>
> > Yi,
> >
> > I can do a curl or telnet to localhost but it is hard to simulate as it
> > spawns random ports.
> > The job fails.
> >
> > Here is the application log:
> >
> > 2016-08-04 20:40:56 SamzaAppMaster$ [INFO] got container id:
> > container_1470270235966_0016_02_000001
> >
> > 2016-08-04 20:40:56 SamzaAppMaster$ [INFO] got app attempt id:
> > appattempt_1470270235966_0016_000002
> >
> > 2016-08-04 20:40:56 SamzaAppMaster$ [INFO] got node manager host:
> > sprdargas102
> >
> > 2016-08-04 20:40:56 SamzaAppMaster$ [INFO] got node manager port: 40531
> >
> > 2016-08-04 20:40:56 SamzaAppMaster$ [INFO] got node manager http port:
> 8042
> >
> > 2016-08-04 20:40:57 SamzaAppMaster$ [INFO] got coordinator system config:
> > {systems.kafka.consumer.zookeeper.connect=sprdargas102:2181,
> > sprdargas103:2181,sprdargas104:2181, systems.kafka.samza.factory=
> > org.apache.samza.system.kafka.KafkaSystemFactory,
> > job.coordinator.system=kafka, systems.kafka.streams.metrics.
> samza.msg.serde=metrics,
> > job.id=1, systems.kafka.producer.producer.type=sync,
> > systems.kafka.producer.metadata.broker.list=sprdargas102:6667,
> sprdargas103:6667,
> > systems.kafka.producer.batch.num.messages=1,
> > systems.kafka.samza.msg.serde=json, systems.kafka.consumer.auto.
> offset.reset=largest,
> > systems.kafka.streams.PageViewEvent.samza.msg.serde=json,
> > systems.kafka.streams.ArgosRaw.samza.msg.serde=json, job.name=Argos,
> > systems.kafka.producer.bootstrap.servers=sprdargas102:6667,
> > sprdargas103:6667}
> >
> > 2016-08-04 20:40:57 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> auto.offset.reset
> > is overridden to largest
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343257174-0
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property group.id is
> > overridden to undefined-samza-consumer-group-427db503-329d-4201-87da-
> > eed83f9f1390
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> zookeeper.connect
> > is overridden to sprdargas102:2181,sprdargas103:2181,sprdargas104:2181
> >
> > 2016-08-04 20:40:57 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> auto.offset.reset
> > is overridden to largest
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_consumer-Argos-1-1470343257242-1
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property group.id is
> > overridden to undefined-samza-consumer-group-aa7250b3-ffa6-4caa-8952-
> > 6aaaa8c275f1
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> zookeeper.connect
> > is overridden to sprdargas102:2181,sprdargas103:2181,sprdargas104:2181
> >
> > 2016-08-04 20:40:57 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> auto.offset.reset
> > is overridden to largest
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343257270-2
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property group.id is
> > overridden to undefined-samza-consumer-group-00ed785c-1ee2-41cd-82ec-
> > 78c7a5e4f404
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> zookeeper.connect
> > is overridden to sprdargas102:2181,sprdargas103:2181,sprdargas104:2181
> >
> > 2016-08-04 20:40:57 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> auto.offset.reset
> > is overridden to largest
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343257283-3
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property group.id is
> > overridden to undefined-samza-consumer-group-d75f6419-4985-45d1-aeed-
> > 18351ca05be5
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> zookeeper.connect
> > is overridden to sprdargas102:2181,sprdargas103:2181,sprdargas104:2181
> >
> > 2016-08-04 20:40:57 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:57 JobCoordinator$ [INFO] Registering coordinator system
> > stream.
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343257174-0
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> > metadata.broker.list is overridden to sprdargas102:6667,
> sprdargas103:6667
> >
> > 2016-08-04 20:40:57 VerifiableProperties [INFO] Property
> > request.timeout.ms is overridden to 30000
> >
> > 2016-08-04 20:40:57 ClientUtils$ [INFO] Fetching metadata from broker
> > id:0,host:sprdargas102,port:6667 with correlation id 0 for 1 topic(s)
> > Set(__samza_coordinator_Argos_1)
> >
> > 2016-08-04 20:40:57 SyncProducer [INFO] Connected to sprdargas102:6667
> for
> > producing
> >
> > 2016-08-04 20:40:57 SyncProducer [INFO] Disconnecting from
> > sprdargas102:6667
> >
> > 2016-08-04 20:40:57 KafkaSystemAdmin$ [INFO] Got metadata:
> > Map(__samza_coordinator_Argos_1 -> SystemStreamMetadata
> > [streamName=__samza_coordinator_Argos_1, partitionMetadata={Partition
> > [partition=0]=SystemStreamPartitionMetadata [oldestOffset=0,
> > newestOffset=1269, upcomingOffset=1270]}])
> >
> > 2016-08-04 20:40:57 CoordinatorStreamSystemConsumer [INFO] Starting
> > coordinator stream system consumer.
> >
> > 2016-08-04 20:40:57 KafkaSystemConsumer [INFO] Refreshing brokers for:
> > Map([__samza_coordinator_Argos_1,0] -> 0)
> >
> > 2016-08-04 20:40:57 BrokerProxy [INFO] Creating new SimpleConsumer for
> > host sprdargas102:6667 for system kafka
> >
> > 2016-08-04 20:40:57 GetOffset [INFO] Validating offset 0 for topic and
> > partition [__samza_coordinator_Argos_1,0]
> >
> > 2016-08-04 20:40:57 GetOffset [INFO] Able to successfully read from
> offset
> > 0 for topic and partition [__samza_coordinator_Argos_1,0]. Using it to
> > instantiate consumer.
> >
> > 2016-08-04 20:40:57 BrokerProxy [INFO] Starting BrokerProxy for
> > sprdargas102:6667
> >
> > 2016-08-04 20:40:57 CoordinatorStreamSystemConsumer [INFO] Bootstrapping
> > configuration from coordinator stream.
> >
> > 2016-08-04 20:40:58 JobCoordinator$ [INFO] Got config:
> > {systems.kafka.samza.factory=org.apache.samza.system.kafka.
> KafkaSystemFactory,
> > system.redis=sprdargas104, task.drop.deserialization.errors=true,
> > serializers.registry.metrics.class=org.apache.samza.serializers.
> MetricsSnapshotSerdeFactory,
> > systems.kafka.streams.metrics.samza.msg.serde=metrics,
> > serializers.registry.string.class=org.apache.samza.serializers.
> StringSerdeFactory,
> > task.checkpoint.system=kafka, task.checkpoint.factory=org.
> > apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory,
> > task.ignored.exceptions=org.codehaus.jackson.JsonParseException,
> > systems.kafka.producer.metadata.broker.list=sprdargas102:6667,
> sprdargas103:6667,
> > systems.kafka.samza.msg.serde=json, metrics.reporters=snapshot,jmx,
> > systems.kafka.consumer.auto.offset.reset=largest, job.name=Argos,
> > metrics.reporter.snapshot.class=org.apache.samza.metrics.reporter.
> > MetricsSnapshotReporterFactory, systems.kafka.producer.
> bootstrap.servers=
> > sprdargas102:6667,sprdargas103:6667, task.checkpoint.replication.
> factor=2,
> > systems.kafka.consumer.zookeeper.connect=sprdargas102:2181,
> > sprdargas103:2181,sprdargas104:2181, load.property=true,
> > job.coordinator.system=kafka, task.inputs=kafka.argos-raw,
> > systems.kafka.producer.producer.type=sync, task.outputs=argos-parser,
> > yarn.package.path=http://sprdargas102:8000/argos_aggregate.tar.gz,
> > job.factory.class=org.apache.samza.job.yarn.YarnJobFactory,
> > task.class=com.intuit.argos.task.ArgosParserStreamTask,
> > redis.wildcard.lookup=true, metrics.reporter.jmx.class=
> > org.apache.samza.metrics.reporter.JmxReporterFactory,
> > systems.kafka.producer.batch.num.messages=1, systems.kafka.streams.
> > PageViewEvent.samza.msg.serde=json, task.drop.serialization.errors=true,
> > systems.kafka.streams.ArgosRaw.samza.msg.serde=json,
> > serializers.registry.json.class=org.apache.samza.
> serializers.JsonSerdeFactory,
> > task.opts=-Dcom.sun.management.jmxremote -Dcom.sun.management.
> jmxremote.port=1099
> > -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.
> jmxremote.ssl=false
> > -Djava.rmi.server.hostname=127.0.0.1, metrics.reporter.snapshot.
> > stream=kafka.metrics}
> >
> > 2016-08-04 20:40:58 KafkaProducerConfig [WARN] Kafka producer
> > configuration contains 'metadata.broker.list'. This configuration is
> > deprecated . Samza has been upgraded to use Kafka's new producer API.
> > Please update your configurations based on the documentation at
> > http://kafka.apache.org/documentation.html#newproducerconfigs
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property
> auto.offset.reset
> > is overridden to largest
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343258775-5
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property group.id is
> > overridden to undefined-samza-consumer-group-b18c1f75-fd2a-4eea-bb28-
> > 98d5a85fc3c8
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property
> zookeeper.connect
> > is overridden to sprdargas102:2181,sprdargas103:2181,sprdargas104:2181
> >
> > 2016-08-04 20:40:58 JobConfig [INFO] job.coordinator.replication.factor
> > was not found. Using task.checkpoint.replication.factor=2 for
> coordinator
> > stream
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Verifying properties
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property client.id is
> > overridden to samza_admin-Argos-1-1470343258775-5
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property
> > metadata.broker.list is overridden to sprdargas102:6667,
> sprdargas103:6667
> >
> > 2016-08-04 20:40:58 VerifiableProperties [INFO] Property
> > request.timeout.ms is overridden to 30000
> >
> > 2016-08-04 20:40:58 ClientUtils$ [INFO] Fetching metadata from broker
> > id:1,host:sprdargas103,port:6667 with correlation id 0 for 1 topic(s)
> > Set(argos-raw)
> >
> > 2016-08-04 20:40:58 SyncProducer [INFO] Connected to sprdargas103:6667
> for
> > producing
> >
> > 2016-08-04 20:40:58 SyncProducer [INFO] Disconnecting from
> > sprdargas103:6667
> >
> > 2016-08-04 20:40:58 KafkaSystemAdmin$ [INFO] Got metadata: Map(argos-raw
> > -> SystemStreamMetadata [streamName=argos-raw,
> partitionMetadata={Partition
> > [partition=0]=SystemStreamPartitionMetadata [oldestOffset=0,
> > newestOffset=null, upcomingOffset=0]}])
> >
> > 2016-08-04 20:40:58 JobCoordinator$ [INFO] SystemStreamPartitionGrouper
> > org.apache.samza.container.grouper.stream.GroupByPartition@5674e1f2 has
> > grouped the SystemStreamPartitions into the following taskNames:
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemProducer [INFO] Starting
> > coordinator stream producer.
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemConsumer [INFO] Coordinator
> > stream consumer already started
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemConsumer [INFO] Bootstrapping
> > configuration from coordinator stream.
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemProducer [INFO] Coordinator
> > stream producer already started
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemConsumer [INFO] Coordinator
> > stream consumer already started
> >
> > 2016-08-04 20:40:58 TaskConfig [INFO] No task.name.grouper.factory
> > configuration, using 'org.apache.samza.container.grouper.task.
> > GroupByContainerCountFactory'
> >
> > 2016-08-04 20:40:58 CoordinatorStreamSystemConsumer [INFO] Bootstrapping
> > configuration from coordinator stream.
> >
> > 2016-08-04 20:40:58 JobCoordinator$ [INFO] Saving task-to-changelog
> > partition mapping: Map(Partition 0 -> 0)
> >
> > 2016-08-04 20:40:58 KafkaSystemProducer [INFO] Creating a new producer
> for
> > system kafka.
> >
> > 2016-08-04 20:40:58 ProducerConfig [INFO] ProducerConfig values:
> >
> > compression.type = none
> >
> > metric.reporters = []
> >
> > metadata.max.age.ms = 300000
> >
> > metadata.fetch.timeout.ms = 60000
> >
> > acks = 1
> >
> > batch.size = 16384
> >
> > reconnect.backoff.ms = 10
> >
> > bootstrap.servers = [sprdargas102:6667, sprdargas103:6667]
> >
> > receive.buffer.bytes = 32768
> >
> > retry.backoff.ms = 100
> >
> > buffer.memory = 33554432
> >
> > timeout.ms = 30000
> >
> > key.serializer = class org.apache.kafka.common.serialization.
> > ByteArraySerializer
> >
> > retries = 2147483647
> >
> > max.request.size = 1048576
> >
> > block.on.buffer.full = true
> >
> > value.serializer = class org.apache.kafka.common.serialization.
> > ByteArraySerializer
> >
> > metrics.sample.window.ms = 30000
> >
> > send.buffer.bytes = 131072
> >
> > max.in.flight.requests.per.connection = 1
> >
> > metrics.num.samples = 2
> >
> > linger.ms = 0
> >
> > client.id = samza_producer-Argos-1-1470343257285-4
> >
> >
> > 2016-08-04 20:40:58 ProducerConfig [WARN] The configuration
> > batch.num.messages = null was supplied but isn't a known config.
> >
> > 2016-08-04 20:40:58 ProducerConfig [WARN] The configuration producer.type
> > = null was supplied but isn't a known config.
> >
> > 2016-08-04 20:40:58 ProducerConfig [WARN] The configuration
> > metadata.broker.list = null was supplied but isn't a known config.
> >
> > 2016-08-04 20:40:59 TaskConfig [INFO] No task.name.grouper.factory
> > configuration, using 'org.apache.samza.container.grouper.task.
> > GroupByContainerCountFactory'
> >
> > 2016-08-04 20:40:59 CoordinatorStreamSystemConsumer [INFO] Bootstrapping
> > configuration from coordinator stream.
> >
> > 2016-08-04 20:41:00 JobCoordinator$ [INFO] Got change log stream
> metadata:
> > Map()
> >
> > 2016-08-04 20:41:00 SamzaAppMaster$ [INFO] got config:
> > {systems.kafka.consumer.zookeeper.connect=sprdargas102:2181,
> > sprdargas103:2181,sprdargas104:2181, systems.kafka.samza.factory=
> > org.apache.samza.system.kafka.KafkaSystemFactory,
> > job.coordinator.system=kafka, systems.kafka.streams.metrics.
> samza.msg.serde=metrics,
> > job.id=1, systems.kafka.producer.producer.type=sync,
> > systems.kafka.producer.metadata.broker.list=sprdargas102:6667,
> sprdargas103:6667,
> > systems.kafka.producer.batch.num.messages=1,
> > systems.kafka.samza.msg.serde=json, systems.kafka.consumer.auto.
> offset.reset=largest,
> > systems.kafka.streams.PageViewEvent.samza.msg.serde=json,
> > systems.kafka.streams.ArgosRaw.samza.msg.serde=json, job.name=Argos,
> > systems.kafka.producer.bootstrap.servers=sprdargas102:6667,
> > sprdargas103:6667}
> >
> > 2016-08-04 20:41:00 ClientHelper [INFO] trying to connect to RM
> > 0.0.0.0:8032
> >
> > 2016-08-04 20:41:00 JmxServer [INFO] According to
> > Util.getLocalHost.getHostName we are sprdargas102
> >
> > - Shekar
> >
> > On Thu, Aug 4, 2016 at 2:23 PM, Yi Pan <nickpa...@gmail.com> wrote:
> >
> >> Hi, Shekar,
> >>
> >> Please see my replies below:
> >>
> >> On Thu, Aug 4, 2016 at 12:18 PM, Shekar Tippur <ctip...@gmail.com>
> wrote:
> >>
> >> > Sorry .. got trigger happy.
> >> > Couple of observations:
> >> > 1. rmi port seem to be random ones Even with task.opts and setting the
> >> port
> >> > to 1099
> >> >
> >>
> >> You are right. Samza containers will choose a random port to start the
> >> JmxServer. We do have an issue w/ port conflicts in 0.10.0 and earlier
> >> versions. It is recently fixed in 0.10.1. The 0.10.1 RC is out for the
> >> vote
> >> now. It would be good if you can try it out. However, your exception
> stack
> >> trace indicates that it is not a port conflict, but a refusal of
> >> connection. Did you try telnet to localhost to see whether it can open
> the
> >> connection?
> >>
> >>
> >> > 2. Do I need to start rmiregistry manually?
> >> >
> >> No, you shouldn't need to start any other process manually for that.
> >>
> >> P.S. do you have a full log from your SamzaAppMaster? That would be
> >> helpful
> >> in debugging this issue.
> >>
> >>
> >> > 3. I dont have any firewall settings.
> >> >
> >> > On Thu, Aug 4, 2016 at 12:14 PM, Shekar Tippur <ctip...@gmail.com>
> >> wrote:
> >> >
> >> > > Yi
> >> > > Thanks for responding.. I have tried diff options for task.opts and
> >> > > without it as well.
> >> > > I have tried this
> >> > >
> >> > > task.opts=-Dcom.sun.management.jmxremote -Dcom.sun.management.
> >> > jmxremote.port=1099
> >> > > -Dcom.sun.management.jmxremote.authenticate=false
> >> -Dcom.sun.management.
> >> > jmxremote.ssl=false
> >> > > -Djava.rmi.server.hostname=localhost
> >> > >
> >> > >
> >> > >
> >> > > Here is the property file:
> >> > > ##########################################################
> >> > >
> >> > > # 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.
> >> > >
> >> > >
> >> > > job.factory.class=org.apache.samza.job.yarn.YarnJobFactory
> >> > >
> >> > > job.name=Argos
> >> > >
> >> > > #job.coordinator.system=kafka
> >> > >
> >> > > #job.coordinator.replication.factor=2
> >> > >
> >> > > # YARN
> >> > >
> >> > > yarn.package.path=http://localhost:8000/argos_aggregate.tar.gz
> >> > >
> >> > > #task.opts="-Dcom.sun.management.jmxremote -Dcom.sun.management.
> >> > jmxremote.port=7091
> >> > > -Dcom.sun.management.jmxremote.authenticate=false
> >> > > -Dcom.sun.management.jmxremote.ssl=false
> >> -Djava.rmi.server.hostname=loc
> >> > > alhost"
> >> > >
> >> > > # Task
> >> > >
> >> > > task.class=com.project.argos.task.ArgosParserStreamTask
> >> > >
> >> > > task.inputs=kafka.argos-raw
> >> > >
> >> > > task.outputs=argos-parser
> >> > >
> >> > > task.checkpoint.factory=org.apache.samza.checkpoint.kafka.Ka
> >> > > fkaCheckpointManagerFactory
> >> > >
> >> > > task.checkpoint.system=kafka
> >> > >
> >> > > # Normally, this would be 3, but we have only one broker.
> >> > >
> >> > > task.checkpoint.replication.factor=2
> >> > >
> >> > > task.drop.deserialization.errors=true
> >> > >
> >> > > task.drop.serialization.errors=true
> >> > >
> >> > > task.ignored.exceptions=org.codehaus.jackson.JsonParseException
> >> > >
> >> > >
> >> > > # Samza10 properties
> >> > >
> >> > > #systems.kafka.producer.bootstrap.servers=localhost:6667
> >> > >
> >> > > systems.kafka.producer.bootstrap.servers=host1:6667,host1:6667
> >> > >
> >> > > job.coordinator.system=kafka
> >> > >
> >> > >
> >> > >
> >> > > # Metrics
> >> > >
> >> > > metrics.reporters=snapshot,jmx
> >> > >
> >> > > metrics.reporter.snapshot.class=org.apache.samza.metrics.
> >> > > reporter.MetricsSnapshotReporterFactory
> >> > >
> >> > > metrics.reporter.snapshot.stream=kafka.metrics
> >> > >
> >> > > metrics.reporter.jmx.class=org.apache.samza.metrics.reporter
> >> > > .JmxReporterFactory
> >> > >
> >> > >
> >> > > # Serializers
> >> > >
> >> > > serializers.registry.json.class=org.apache.samza.serializers
> >> > > .JsonSerdeFactory
> >> > >
> >> > > serializers.registry.string.class=org.apache.samza.serialize
> >> > > rs.StringSerdeFactory
> >> > >
> >> > > serializers.registry.metrics.class=org.apache.samza.serializ
> >> > > ers.MetricsSnapshotSerdeFactory
> >> > >
> >> > >
> >> > > # Systems
> >> > >
> >> > > systems.kafka.samza.factory=org.apache.samza.system.kafka.Ka
> >> > > fkaSystemFactory
> >> > >
> >> > > systems.kafka.samza.msg.serde=json
> >> > >
> >> > >
> >> > > systems.kafka.consumer.zookeeper.connect=host1:2181,
> >> > host2:2181,host3:2181
> >> > >
> >> > > systems.kafka.consumer.auto.offset.reset=largest
> >> > >
> >> > > systems.kafka.producer.producer.type=sync
> >> > >
> >> > > # Normally, we'd set this much higher, but we want things to look
> >> snappy
> >> > > in the demo.
> >> > >
> >> > > systems.kafka.producer.batch.num.messages=1
> >> > >
> >> > > systems.kafka.streams.metrics.samza.msg.serde=metrics
> >> > >
> >> > > systems.kafka.streams.ArgosRaw.samza.msg.serde=json
> >> > >
> >> > > systems.kafka.streams.PageViewEvent.samza.msg.serde=json
> >> > >
> >> > >
> >> > >
> >> > > # enable wildcard lookup on cache misses, will impact performance
> >> > >
> >> > > redis.wildcard.lookup=true
> >> > >
> >> > >
> >> > > # Property File
> >> > >
> >> > > load.property=true
> >> > >
> >> > >
> >> > >
> >> > > system.redis=host4
> >> > >
> >> > > systems.kafka.producer.metadata.broker.list=host1:6667host2:6667
> >> > >
> >> > > ##########################################################
> >> > >
> >> > > On Thu, Aug 4, 2016 at 11:19 AM, Yi Pan <nickpa...@gmail.com>
> wrote:
> >> > >
> >> > >> Hi, Shekar,
> >> > >>
> >> > >> Did you check your firewall configuration? Could you also paste
> your
> >> > >> configuration, especially task.opts?
> >> > >>
> >> > >> -Yi
> >> > >>
> >> > >> On Wed, Aug 3, 2016 at 5:56 PM, Shekar Tippur <ctip...@gmail.com>
> >> > wrote:
> >> > >>
> >> > >> > I am trying to submit a Samza job to yarn and I get a error:
> >> > >> >
> >> > >> > Exception in thread "main" java.io.IOException: Cannot bind to
> URL
> >> > >> > [rmi://localhost:44960/jmxrmi]: javax.naming.ServiceUnavailabl
> >> > >> eException
> >> > >> > [Root exception is java.rmi.ConnectException: Connection refused
> to
> >> > >> host:
> >> > >> > localhost; nested exception is:
> >> > >> >
> >> > >> >         java.net.ConnectException: Connection refused]
> >> > >> >
> >> > >> >         at
> >> > >> > javax.management.remote.rmi.RMIConnectorServer.newIOException(
> >> > >> > RMIConnectorServer.java:827)
> >> > >> >
> >> > >> >         at
> >> > >> > javax.management.remote.rmi.RMIConnectorServer.start(
> >> > >> > RMIConnectorServer.java:432)
> >> > >> >
> >> > >> >         at org.apache.samza.metrics.JmxServer.<init>(JmxServer.
> >> > scala:89
> >> > >> )
> >> > >> >
> >> > >> >         at org.apache.samza.metrics.JmxServer.<init>(JmxServer.
> >> > scala:43
> >> > >> )
> >> > >> >
> >> > >> >         at
> >> > >> > org.apache.samza.job.yarn.SamzaAppMaster$.main(
> >> > SamzaAppMaster.scala:87)
> >> > >> >
> >> > >> >         at
> >> > >> > org.apache.samza.job.yarn.SamzaAppMaster.main(SamzaAppMaster
> >> .scala)
> >> > >> >
> >> > >> > Caused by: javax.naming.ServiceUnavailableException [Root
> >> exception
> >> > is
> >> > >> > java.rmi.ConnectException: Connection refused to host: localhost;
> >> > nested
> >> > >> > exception is:
> >> > >> >
> >> > >> >         java.net.ConnectException: Connection refused]
> >> > >> >
> >> > >> >         at
> >> > >> > com.sun.jndi.rmi.registry.RegistryContext.bind(RegistryConte
> >> > >> xt.java:147)
> >> > >> >
> >> > >> >         at
> >> > >> > com.sun.jndi.toolkit.url.GenericURLContext.bind(
> >> > >> > GenericURLContext.java:228)
> >> > >> >
> >> > >> >         at javax.naming.InitialContext.bi
> >> nd(InitialContext.java:425)
> >> > >> >
> >> > >> >         at
> >> > >> > javax.management.remote.rmi.RMIConnectorServer.bind(
> >> > >> > RMIConnectorServer.java:644)
> >> > >> >
> >> > >> >         at
> >> > >> > javax.management.remote.rmi.RMIConnectorServer.start(
> >> > >> > RMIConnectorServer.java:427)
> >> > >> >
> >> > >> >         ... 4 more
> >> > >> >
> >> > >> > Caused by: java.rmi.ConnectException: Connection refused to host:
> >> > >> > localhost; nested exception is:
> >> > >> >
> >> > >> >         java.net.ConnectException: Connection refused
> >> > >> >
> >> > >> >         at sun.rmi.transport.tcp.TCPEndpoint.newSocket(
> >> > >> > TCPEndpoint.java:619)
> >> > >> >
> >> > >> >         at
> >> > >> > sun.rmi.transport.tcp.TCPChannel.createConnection(
> >> > TCPChannel.java:216)
> >> > >> >
> >> > >> >         at
> >> > >> > sun.rmi.transport.tcp.TCPChannel.newConnection(TCPChannel.
> >> java:202)
> >> > >> >
> >> > >> >         at sun.rmi.server.UnicastRef.
> newCall(UnicastRef.java:342)
> >> > >> >
> >> > >> >         at sun.rmi.registry.RegistryImpl_Stub.bind(Unknown
> Source)
> >> > >> >
> >> > >> >         at
> >> > >> > com.sun.jndi.rmi.registry.RegistryContext.bind(RegistryConte
> >> > >> xt.java:141)
> >> > >> >
> >> > >> >         ... 8 more
> >> > >> >
> >> > >> > Caused by: java.net.ConnectException: Connection refused
> >> > >> >
> >> > >> >         at java.net.PlainSocketImpl.socketConnect(Native Method)
> >> > >> >
> >> > >> >         at
> >> > >> > java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSock
> >> > >> etImpl.java:
> >> > >> > 350)
> >> > >> >
> >> > >> >         at
> >> > >> > java.net.AbstractPlainSocketImpl.connectToAddress(
> >> > >> > AbstractPlainSocketImpl.java:206)
> >> > >> >
> >> > >> >         at
> >> > >> > java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocket
> >> > >> Impl.java:188)
> >> > >> >
> >> > >> >         at java.net.SocksSocketImpl.conne
> >> ct(SocksSocketImpl.java:392)
> >> > >> >
> >> > >> >         at java.net.Socket.connect(Socket.java:589)
> >> > >> >
> >> > >> >         at java.net.Socket.connect(Socket.java:538)
> >> > >> >
> >> > >> >         at java.net.Socket.<init>(Socket.java:434)
> >> > >> >
> >> > >> >         at java.net.Socket.<init>(Socket.java:211)
> >> > >> >
> >> > >> >         at
> >> > >> > sun.rmi.transport.proxy.RMIDirectSocketFactory.createSocket(
> >> > >> > RMIDirectSocketFactory.java:40)
> >> > >> >
> >> > >> >         at
> >> > >> > sun.rmi.transport.proxy.RMIMasterSocketFactory.createSocket(
> >> > >> > RMIMasterSocketFactory.java:148)
> >> > >> >
> >> > >> >         at sun.rmi.transport.tcp.TCPEndpoint.newSocket(
> >> > >> > TCPEndpoint.java:613)
> >> > >> >
> >> > >> >         ... 13 more
> >> > >> >
> >> > >> > Any idea on what could be the issue?
> >> > >> >
> >> > >> > - Shekar
> >> > >> >
> >> > >>
> >> > >
> >> > >
> >> >
> >>
> >
> >
>

Reply via email to