Hi Thunder,

Can you provide debug level logs from KafkaSystemConsumer with the
stack trace for the exception? It'll help figure out why the
auto.offset.reset property isn't taking effect.

If this error is due to an older checkpoint for the stream, you can
try resetting the checkpoint using the following two configurations:
streams.stream-id.samza.reset.offset: If set to true, when a Samza
container starts up, it ignores any checkpointed offset for this
particular input stream. Its behavior is thus determined by the
samza.offset.default setting. Note that the reset takes effect every
time a container is started, which may be every time you restart your
job, or more frequently if a container fails and is restarted by the
framework.

streams.stream-id.samza.offset.default: If a container starts up
without a checkpoint, this property determines where in the input
stream we should start consuming. The value must be an OffsetType, one
of the following:
  upcoming: Start processing messages that are published after the job
starts. Any messages published while the job was not running are not
processed.
  oldest: Start processing at the oldest available message in the
system, and reprocess the entire available message history.

I.e., set 'samza.reset.offset' = true, and 'samza.offset.default' =
oldest for your stream. Let us know if this doesn't help.

Thanks,
Prateek

On Fri, Jul 6, 2018 at 11:43 AM, Thunder Stumpges <tstump...@ntent.com> wrote:
> Hi all,
>
>
> We've just run into a strange problem with samza 0.14.1. We had a job down 
> for a bit, while kafka cleaned past our saved offsets. When starting the job 
> now, we get repeated 
> org.apache.kafka.common.errors.OffsetOutOfRangeException. And it just retries 
> over and over again. We HAVE set
>
> systems.kafka.consumer.auto.offset.reset=smallest as well. Has anyone else 
> seen this? Our understanding from the documentation is that this setting says 
> what to do if the offset is out of range.
>
>
>
> systems.system-name.consumer.auto.offset.reset : This setting determines what 
> happens if a consumer attempts to read an offset that is outside of the 
> current valid range. This could happen if the topic does not exist, or if a 
> checkpoint is older than the maximum message history retained by the brokers.
>
>
>
> This is the set of messages that keeps repeating:
>
>
>
> 2018-07-06 18:32:15 INFO  kafka.utils.VerifiableProperties - Verifying 
> properties
>
> 2018-07-06 18:32:15 INFO  kafka.utils.VerifiableProperties - Property 
> client.id is overridden to samza_consumer-stg_apollo_crawler_stream_task-1
>
> 2018-07-06 18:32:15 INFO  kafka.utils.VerifiableProperties - Property 
> metadata.broker.list is overridden to kafka-server.ntent.com:9092
>
> 2018-07-06 18:32:15 INFO  kafka.utils.VerifiableProperties - Property 
> request.timeout.ms is overridden to 30000
>
> 2018-07-06 18:32:15 INFO  kafka.client.ClientUtils$ - Fetching metadata from 
> broker BrokerEndPoint(0,kafka-server,9092) with correlation id 12 for 1 
> topic(s) Set(my-topic)
>
> 2018-07-06 18:32:15 INFO  kafka.producer.SyncProducer - Connected to 
> kafka-server:9092 for producing
>
> 2018-07-06 18:32:15 INFO  kafka.producer.SyncProducer - Disconnecting from 
> kafka-server:9092
>
> 2018-07-06 18:32:15 INFO  o.a.samza.system.kafka.GetOffset - Validating 
> offset 6883929 for topic and partition [my-topic,10]
>
> 2018-07-06 18:32:15 WARN  o.a.s.s.kafka.KafkaSystemConsumer - While 
> refreshing brokers for [my-topic,10]: 
> org.apache.kafka.common.errors.OffsetOutOfRangeException: The requested 
> offset is not within the range of offsets maintained by the server.. Retrying.
>
>
>
> Thanks!
>
> Thunder
>
>

Reply via email to