Sure, I've attached it to this email. The process seems to restart once the
TimeoutException happens so it's repeated a couple of times.

Thanks for looking at it!

/Tim

On Wed, 11 Nov 2020 at 10:37, Aljoscha Krettek <aljos...@apache.org> wrote:

> Hmm, could you please post the full stack trace that leads to the
> TimeoutException?
>
> Best,
> Aljoscha
>
> On 10.11.20 17:54, Tim Josefsson wrote:
> > Hey Aljoscha,
> >
> > I'm setting the transaction.timeout.ms when I create the
> FlinkKafkaProducer:
> >
> > I create a Properties object and then set the property and finally add
> > those properties when creating the producer.
> >
> > Properties producerProps = new Properties();
> > producerProps.setProperty("transaction.timeout.ms", "900000");
> >
> > If I don't set that property my I instead get the following config when
> > starting the job:
> > 11:41:56,345 INFO  org.apache.kafka.clients.producer.ProducerConfig
> >       - ProducerConfig values:
> >     acks = 1
> >     [omitted for brevity]
> >     transaction.timeout.ms = 60000
> >     transactional.id = Source: Read player events from Kafka -> Map
> >   Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not
> > needed for backfill -> Sink: Post events to playerEvents
> > Kafka-a15b4dd4812495cebdc94e33125ef858-1
> >     value.serializer = class
> > org.apache.kafka.common.serialization.ByteArraySerializer
> >
> > So I imagine the Producer is picking up the change but it still returns
> > errors when running the job.
> >
> > Best regards,
> > Tim
> >
> >
> > On Tue, 10 Nov 2020 at 16:14, Aljoscha Krettek <aljos...@apache.org>
> wrote:
> >
> >> On 10.11.20 11:53, Tim Josefsson wrote:
> >>> Also when checking my logs I see the following message:
> >>> 11:41:56,345 INFO  org.apache.kafka.clients.producer.ProducerConfig
> >>>        - ProducerConfig values:
> >>>      acks = 1
> >>>      [omitted for brevity]
> >>>      transaction.timeout.ms = 900000
> >>>      transactional.id = Source: Read player events from Kafka -> Map
> >>>    Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates
> not
> >>> needed for backfill -> Sink: Post events to playerEvents
> >>> Kafka-a15b4dd4812495cebdc94e33125ef858-1
> >>>      value.serializer = class
> >>> org.apache.kafka.common.serialization.ByteArraySerializer
> >>
> >> The interesting thing would be to figure out where that
> >> `transaction.timeout.ms = 900000` is coming from. The default from
> Flink
> >> would be 60000, if nothing is configured. Are you specifying that value,
> >> maybe from the commandline or in code?
> >>
> >> Maybe it's a funny coincidence, but our StreamingKafkaITCase also
> >> specifies that timeout value.
> >>
> >> Best,
> >> Aljoscha
> >>
> >>
> >
>
>

-- 

*Tim Josefsson*
[image: Webstep GPtW] <http://www.webstep.se/>
mobil   +46 (0) 707 81 91 12
telefon +46 (0) 8 21 40 70

tim.josefs...@webstep.se
*webstep.se <http://www.webstep.se/>*
Suttungs gränd 2
753 19 Uppsala
Stockholm | Uppsala | Malmö | Sundsvall | Oslo
Bergen | Stavanger | Trondheim | Kristiansand
[image: LinkedIn] <http://www.linkedin.com/company/webstep-ab> [image:
Facebook] <http://www.facebook.com/webstepAB> [image: Facebook]
<http://www.instagram.com/webstep_sverige>
13:30:10,884 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Starting Flink Mini Cluster
13:30:10,886 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Starting Metrics Registry
13:30:10,979 INFO  org.apache.flink.runtime.metrics.MetricRegistryImpl           - No metrics reporter configured, no metrics will be exposed/reported.
13:30:10,979 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Starting RPC Service(s)
13:30:11,449 INFO  akka.event.slf4j.Slf4jLogger                                  - Slf4jLogger started
13:30:11,609 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Trying to start actor system at :0
13:30:11,659 INFO  akka.event.slf4j.Slf4jLogger                                  - Slf4jLogger started
13:30:11,677 INFO  akka.remote.Remoting                                          - Starting remoting
13:30:11,816 INFO  akka.remote.Remoting                                          - Remoting started; listening on addresses :[akka.tcp://flink-metrics@127.0.1.1:37125]
13:30:11,859 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcServiceUtils         - Actor system started at akka.tcp://flink-metrics@127.0.1.1:37125
13:30:11,869 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.metrics.dump.MetricQueryService at akka://flink-metrics/user/MetricQueryService .
13:30:11,989 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Starting high-availability services
13:30:12,001 INFO  org.apache.flink.runtime.blob.BlobServer                      - Created BLOB server storage directory /tmp/blobStore-254615eb-1909-4a7a-92a0-a2e9b961b9f5
13:30:12,005 INFO  org.apache.flink.runtime.blob.BlobServer                      - Started BLOB server at 0.0.0.0:46355 - max concurrent requests: 50 - max backlog: 1000
13:30:12,008 INFO  org.apache.flink.runtime.blob.PermanentBlobCache              - Created BLOB cache storage directory /tmp/blobStore-0a33c7a9-e6ea-465e-8aff-8fb71259a998
13:30:12,009 INFO  org.apache.flink.runtime.blob.TransientBlobCache              - Created BLOB cache storage directory /tmp/blobStore-8202de75-ddc9-4b29-a444-023e83432572
13:30:12,010 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Starting 1 TaskManger(s)
13:30:12,012 INFO  org.apache.flink.runtime.taskexecutor.TaskManagerRunner       - Starting TaskManager with ResourceID: bfa9faf5-2a2b-43e0-bef2-c1e4ec2bce3c
13:30:12,019 INFO  org.apache.flink.runtime.taskexecutor.TaskManagerServices     - Temporary file directory '/tmp': total 460 GB, usable 330 GB (71.74% usable)
13:30:12,021 INFO  org.apache.flink.runtime.io.disk.FileChannelManagerImpl       - FileChannelManager uses directory /tmp/flink-io-e55da544-18f9-47d7-bce0-adf64a18b06a for spill files.
13:30:12,027 INFO  org.apache.flink.runtime.io.disk.FileChannelManagerImpl       - FileChannelManager uses directory /tmp/flink-netty-shuffle-faca8d63-e856-4dd6-9caf-a17221675c63 for spill files.
13:30:12,058 INFO  org.apache.flink.runtime.io.network.buffer.NetworkBufferPool  - Allocated 64 MB for network buffer pool (number of memory segments: 2048, bytes per segment: 32768).
13:30:12,063 INFO  org.apache.flink.runtime.io.network.NettyShuffleEnvironment   - Starting the network environment and its components.
13:30:12,064 INFO  org.apache.flink.runtime.taskexecutor.KvStateService          - Starting the kvState service and its components.
13:30:12,070 INFO  org.apache.flink.runtime.taskexecutor.TaskManagerConfiguration  - Messages have a max timeout of 10000 ms
13:30:12,093 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.taskexecutor.TaskExecutor at akka://flink/user/taskmanager_0 .
13:30:12,102 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Start job leader service.
13:30:12,104 INFO  org.apache.flink.runtime.filecache.FileCache                  - User file cache uses directory /tmp/flink-dist-cache-8b9bf259-6000-4b42-a0e1-b95e097ea721
13:30:12,141 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Starting rest endpoint.
13:30:12,395 WARN  org.apache.flink.runtime.webmonitor.WebMonitorUtils           - Log file environment variable 'log.file' is not set.
13:30:12,395 WARN  org.apache.flink.runtime.webmonitor.WebMonitorUtils           - JobManager log files are unavailable in the web dashboard. Log file location not found in environment variable 'log.file' or configuration key 'Key: 'web.log.path' , default: null (fallback keys: [{key=jobmanager.web.log.path, isDeprecated=true}])'.
13:30:12,407 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Failed to load web based job submission extension. Probable reason: flink-runtime-web is not in the classpath.
13:30:12,580 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Rest endpoint listening at localhost:40951
13:30:12,580 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Proposing leadership to contender http://localhost:40951
13:30:12,582 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - http://localhost:40951 was granted leadership with leaderSessionID=2c0a511a-232b-4cea-a407-c43b35073b91
13:30:12,583 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Received confirmation of leadership for leader http://localhost:40951 , session=2c0a511a-232b-4cea-a407-c43b35073b91
13:30:12,593 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.resourcemanager.StandaloneResourceManager at akka://flink/user/resourcemanager .
13:30:12,608 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Proposing leadership to contender LeaderContender: DefaultDispatcherRunner
13:30:12,609 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Proposing leadership to contender LeaderContender: StandaloneResourceManager
13:30:12,611 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - ResourceManager akka://flink/user/resourcemanager was granted leadership with fencing token 871df91788451578f9dba01d99664c45
13:30:12,612 INFO  org.apache.flink.runtime.minicluster.MiniCluster              - Flink Mini Cluster started successfully
13:30:12,616 INFO  org.apache.flink.runtime.resourcemanager.slotmanager.SlotManagerImpl  - Starting the SlotManager.
13:30:12,618 INFO  org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess  - Start SessionDispatcherLeaderProcess.
13:30:12,619 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Received confirmation of leadership for leader akka://flink/user/resourcemanager , session=f9dba01d-9966-4c45-871d-f91788451578
13:30:12,621 INFO  org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess  - Recover all persisted job graphs.
13:30:12,625 INFO  org.apache.flink.runtime.dispatcher.runner.SessionDispatcherLeaderProcess  - Successfully recovered 0 persisted job graphs.
13:30:12,626 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Connecting to ResourceManager akka://flink/user/resourcemanager(871df91788451578f9dba01d99664c45).
13:30:12,634 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.dispatcher.StandaloneDispatcher at akka://flink/user/dispatcher .
13:30:12,641 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Resolved ResourceManager address, beginning registration
13:30:12,642 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Registration at ResourceManager attempt 1 (timeout=100ms)
13:30:12,650 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Received confirmation of leadership for leader akka://flink/user/dispatcher , session=7122ea8d-5fec-4084-a7d1-a256a561e818
13:30:12,651 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering TaskManager with ResourceID bfa9faf5-2a2b-43e0-bef2-c1e4ec2bce3c (akka://flink/user/taskmanager_0) at ResourceManager
13:30:12,655 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Successful registration at resource manager akka://flink/user/resourcemanager under registration id 539d81ce4683a3a1549deb231ade5207.
13:30:12,674 INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Received JobGraph submission ca9c3d38f9f1f64902dccf832dd51c79 (Transfer Processor).
13:30:12,674 INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher      - Submitting job ca9c3d38f9f1f64902dccf832dd51c79 (Transfer Processor).
13:30:12,696 INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService              - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.JobMaster at akka://flink/user/jobmanager_1 .
13:30:12,705 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Initializing job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79).
13:30:12,724 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Using restart back off time strategy FailureRateRestartBackoffTimeStrategy(FailureRateRestartBackoffTimeStrategy(failuresIntervalMS=3600000,backoffTimeMS=60000,maxFailuresPerInterval=60) for Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79).
13:30:12,771 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Running initialization on master for job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79).
13:30:12,771 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Successfully ran initialization on master in 0 ms.
13:30:12,802 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - No state backend has been configured, using default (Memory / JobManager) MemoryStateBackend (data in heap memory / checkpoints to JobManager) (checkpoints: 'null', savepoints: 'null', asynchronous: TRUE, maxStateSize: 5242880)
13:30:12,816 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - Start building failover regions.
13:30:12,817 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - Created 1 failover regions.
13:30:12,817 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Using failover strategy org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy@4304717c for Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79).
13:30:12,820 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Proposing leadership to contender akka://flink/user/jobmanager_1
13:30:12,821 INFO  org.apache.flink.runtime.jobmaster.JobManagerRunnerImpl       - JobManager runner for job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) was granted leadership with session id 47705458-a079-440b-8f10-ed43f9c0eb40 at akka://flink/user/jobmanager_1.
13:30:12,825 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Starting execution of job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) under job master id 8f10ed43f9c0eb4047705458a079440b.
13:30:12,828 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Starting scheduling with scheduling strategy [org.apache.flink.runtime.scheduler.strategy.EagerSchedulingStrategy]
13:30:12,828 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) switched from state CREATED to RUNNING.
13:30:12,837 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from CREATED to SCHEDULED.
13:30:12,852 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{9491c4f58e044097df4243c94c3fb413}]
13:30:12,859 INFO  org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedLeaderService  - Received confirmation of leadership for leader akka://flink/user/jobmanager_1 , session=47705458-a079-440b-8f10-ed43f9c0eb40
13:30:12,860 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Connecting to ResourceManager akka://flink/user/resourcemanager(871df91788451578f9dba01d99664c45)
13:30:12,862 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Resolved ResourceManager address, beginning registration
13:30:12,862 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Registration at ResourceManager attempt 1 (timeout=100ms)
13:30:12,864 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registering job manager 8f10ed43f9c0eb4047705458a079440b@akka://flink/user/jobmanager_1 for job ca9c3d38f9f1f64902dccf832dd51c79.
13:30:12,871 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Registered job manager 8f10ed43f9c0eb4047705458a079440b@akka://flink/user/jobmanager_1 for job ca9c3d38f9f1f64902dccf832dd51c79.
13:30:12,873 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager successfully registered at ResourceManager, leader id: 871df91788451578f9dba01d99664c45.
13:30:12,874 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Requesting new slot [SlotRequestId{9491c4f58e044097df4243c94c3fb413}] and profile ResourceProfile{UNKNOWN} from resource manager.
13:30:12,875 INFO  org.apache.flink.runtime.resourcemanager.StandaloneResourceManager  - Request slot with profile ResourceProfile{UNKNOWN} for job ca9c3d38f9f1f64902dccf832dd51c79 with allocation id e0de371ec52f37468c9ba3973089780c.
13:30:12,879 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Receive slot request e0de371ec52f37468c9ba3973089780c for job ca9c3d38f9f1f64902dccf832dd51c79 from resource manager with leader id 871df91788451578f9dba01d99664c45.
13:30:12,893 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Allocated slot for e0de371ec52f37468c9ba3973089780c.
13:30:12,893 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Add job ca9c3d38f9f1f64902dccf832dd51c79 for job leader monitoring.
13:30:12,895 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Try to register at job manager akka://flink/user/jobmanager_1 with leader id 47705458-a079-440b-8f10-ed43f9c0eb40.
13:30:12,896 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Resolved JobManager address, beginning registration
13:30:12,896 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Registration at JobManager attempt 1 (timeout=100ms)
13:30:12,899 INFO  org.apache.flink.runtime.taskexecutor.JobLeaderService        - Successful registration at job manager akka://flink/user/jobmanager_1 for job ca9c3d38f9f1f64902dccf832dd51c79.
13:30:12,900 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Establish JobManager connection for job ca9c3d38f9f1f64902dccf832dd51c79.
13:30:12,903 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Offer reserved slots to the leader of job ca9c3d38f9f1f64902dccf832dd51c79.
13:30:12,910 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from SCHEDULED to DEPLOYING.
13:30:12,910 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (attempt #0) to bfa9faf5-2a2b-43e0-bef2-c1e4ec2bce3c @ localhost (dataPort=-1)
13:30:12,943 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Received task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1).
13:30:12,944 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from CREATED to DEPLOYING.
13:30:12,945 INFO  org.apache.flink.runtime.taskmanager.Task                     - Creating FileSystem stream leak safety net for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) [DEPLOYING]
13:30:12,949 INFO  org.apache.flink.runtime.taskexecutor.slot.TaskSlotTableImpl  - Activate slot e0de371ec52f37468c9ba3973089780c.
13:30:12,949 INFO  org.apache.flink.runtime.taskmanager.Task                     - Loading JAR files for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) [DEPLOYING].
13:30:12,952 INFO  org.apache.flink.runtime.taskmanager.Task                     - Registering task at network: Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) [DEPLOYING].
13:30:12,965 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from DEPLOYING to RUNNING.
13:30:12,967 INFO  org.apache.flink.streaming.runtime.tasks.StreamTask           - No state backend has been configured, using default (Memory / JobManager) MemoryStateBackend (data in heap memory / checkpoints to JobManager) (checkpoints: 'null', savepoints: 'null', asynchronous: TRUE, maxStateSize: 5242880)
13:30:12,967 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from DEPLOYING to RUNNING.
13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,052 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:30:13,090 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-5, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Instantiated a transactional producer.
13:30:13,090 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-7, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Instantiated a transactional producer.
13:30:13,090 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-6, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Instantiated a transactional producer.
13:30:13,091 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-4, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Instantiated a transactional producer.
13:30:13,091 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-3, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Instantiated a transactional producer.
13:30:13,091 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-1, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Instantiated a transactional producer.
13:30:13,090 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-2, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Instantiated a transactional producer.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-4, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-2, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-7, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-5, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-5, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Overriding the default acks to all since idempotence is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-6, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-3, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-6, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Overriding the default acks to all since idempotence is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-7, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Overriding the default acks to all since idempotence is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-1, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-2, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Overriding the default acks to all since idempotence is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-4, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Overriding the default acks to all since idempotence is enabled.
13:30:13,168 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-1, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Overriding the default acks to all since idempotence is enabled.
13:30:13,167 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-3, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Overriding the default acks to all since idempotence is enabled.
13:30:13,190 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,190 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,193 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,193 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,194 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,195 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,195 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-6, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] ProducerId set to -1 with epoch -1
13:30:13,195 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-5, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] ProducerId set to -1 with epoch -1
13:30:13,195 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-1, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] ProducerId set to -1 with epoch -1
13:30:13,195 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,196 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,196 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-3, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] ProducerId set to -1 with epoch -1
13:30:13,196 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,196 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,197 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-2, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] ProducerId set to -1 with epoch -1
13:30:13,197 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,197 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,198 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-4, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] ProducerId set to -1 with epoch -1
13:30:13,198 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:30:13,198 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:30:13,199 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-7, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] ProducerId set to -1 with epoch -1
13:30:13,485 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,488 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,489 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,489 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,488 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,491 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:13,491 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:30:19,129 INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator     - Triggering checkpoint 1 @ 1605184219116 for job ca9c3d38f9f1f64902dccf832dd51c79.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-6, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-5, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-4, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-1, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-3, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-7, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,324 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-2, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:31:13,356 INFO  com.ebs.flink.transferprocessor.utils.DateFilter              - Shutting down
13:31:13,359 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from RUNNING to FAILED.
org.apache.kafka.common.errors.TimeoutException: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
Caused by: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
13:31:13,360 INFO  org.apache.flink.runtime.taskmanager.Task                     - Freeing task resources for Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf).
13:31:13,360 INFO  org.apache.flink.runtime.taskmanager.Task                     - Ensuring all FileSystem streams are closed for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) [FAILED]
13:31:13,369 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Un-registering task and sending final execution state FAILED to JobManager for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) fc01c604d3145543d3c73b0730e888bf.
13:31:13,371 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (fc01c604d3145543d3c73b0730e888bf) switched from RUNNING to FAILED.
org.apache.kafka.common.errors.TimeoutException: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
Caused by: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
13:31:13,375 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - Calculating tasks to restart to recover the failed task fce6063a4f1b368072cffb5f5909e445_0.
13:31:13,375 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - 1 tasks should be restarted to recover the failed task fce6063a4f1b368072cffb5f5909e445_0. 
13:31:13,376 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) switched from state RUNNING to RESTARTING.
13:32:13,381 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) switched from state RESTARTING to RUNNING.
13:32:13,391 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from CREATED to SCHEDULED.
13:32:13,397 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from SCHEDULED to DEPLOYING.
13:32:13,398 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (attempt #1) to bfa9faf5-2a2b-43e0-bef2-c1e4ec2bce3c @ localhost (dataPort=-1)
13:32:13,401 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Received task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1).
13:32:13,402 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from CREATED to DEPLOYING.
13:32:13,403 INFO  org.apache.flink.runtime.taskmanager.Task                     - Creating FileSystem stream leak safety net for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) [DEPLOYING]
13:32:13,404 INFO  org.apache.flink.runtime.taskmanager.Task                     - Loading JAR files for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) [DEPLOYING].
13:32:13,407 INFO  org.apache.flink.runtime.taskmanager.Task                     - Registering task at network: Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) [DEPLOYING].
13:32:13,409 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from DEPLOYING to RUNNING.
13:32:13,410 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from DEPLOYING to RUNNING.
13:32:13,411 INFO  org.apache.flink.streaming.runtime.tasks.StreamTask           - No state backend has been configured, using default (Memory / JobManager) MemoryStateBackend (data in heap memory / checkpoints to JobManager) (checkpoints: 'null', savepoints: 'null', asynchronous: TRUE, maxStateSize: 5242880)
13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,439 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-11, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Instantiated a transactional producer.
13:32:13,431 INFO  org.apache.kafka.clients.producer.ProducerConfig              - ProducerConfig values: 
	acks = 1
	batch.size = 16384
	bootstrap.servers = [hidden]
	buffer.memory = 33554432
	client.dns.lookup = default
	client.id = 
	compression.type = none
	connections.max.idle.ms = 540000
	delivery.timeout.ms = 120000
	enable.idempotence = false
	interceptor.classes = []
	key.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer
	linger.ms = 0
	max.block.ms = 60000
	max.in.flight.requests.per.connection = 5
	max.request.size = 1048576
	metadata.max.age.ms = 300000
	metric.reporters = []
	metrics.num.samples = 2
	metrics.recording.level = INFO
	metrics.sample.window.ms = 30000
	partitioner.class = class org.apache.kafka.clients.producer.internals.DefaultPartitioner
	receive.buffer.bytes = 32768
	reconnect.backoff.max.ms = 1000
	reconnect.backoff.ms = 50
	request.timeout.ms = 30000
	retries = 2147483647
	retry.backoff.ms = 100
	sasl.client.callback.handler.class = null
	sasl.jaas.config = null
	sasl.kerberos.kinit.cmd = /usr/bin/kinit
	sasl.kerberos.min.time.before.relogin = 60000
	sasl.kerberos.service.name = null
	sasl.kerberos.ticket.renew.jitter = 0.05
	sasl.kerberos.ticket.renew.window.factor = 0.8
	sasl.login.callback.handler.class = null
	sasl.login.class = null
	sasl.login.refresh.buffer.seconds = 300
	sasl.login.refresh.min.period.seconds = 60
	sasl.login.refresh.window.factor = 0.8
	sasl.login.refresh.window.jitter = 0.05
	sasl.mechanism = GSSAPI
	security.protocol = PLAINTEXT
	send.buffer.bytes = 131072
	ssl.cipher.suites = null
	ssl.enabled.protocols = [TLSv1.2, TLSv1.1, TLSv1]
	ssl.endpoint.identification.algorithm = https
	ssl.key.password = null
	ssl.keymanager.algorithm = SunX509
	ssl.keystore.location = null
	ssl.keystore.password = null
	ssl.keystore.type = JKS
	ssl.protocol = TLS
	ssl.provider = null
	ssl.secure.random.implementation = null
	ssl.trustmanager.algorithm = PKIX
	ssl.truststore.location = null
	ssl.truststore.password = null
	ssl.truststore.type = JKS
	transaction.timeout.ms = 900000
	transactional.id = Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9
	value.serializer = class org.apache.kafka.common.serialization.ByteArraySerializer

13:32:13,439 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-12, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Instantiated a transactional producer.
13:32:13,439 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-13, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Instantiated a transactional producer.
13:32:13,439 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-9, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Instantiated a transactional producer.
13:32:13,439 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-10, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Instantiated a transactional producer.
13:32:13,436 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-8, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Instantiated a transactional producer.
13:32:13,445 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-14, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Instantiated a transactional producer.
13:32:13,453 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-14, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,453 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-14, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Overriding the default acks to all since idempotence is enabled.
13:32:13,463 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-10, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,463 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-10, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Overriding the default acks to all since idempotence is enabled.
13:32:13,464 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,465 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,466 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-10, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] ProducerId set to -1 with epoch -1
13:32:13,470 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-13, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,470 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-13, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Overriding the default acks to all since idempotence is enabled.
13:32:13,470 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-12, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,470 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-12, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Overriding the default acks to all since idempotence is enabled.
13:32:13,472 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,476 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,477 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-14, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] ProducerId set to -1 with epoch -1
13:32:13,481 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,481 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-11, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,482 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-11, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Overriding the default acks to all since idempotence is enabled.
13:32:13,482 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,481 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-8, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,484 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-8, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Overriding the default acks to all since idempotence is enabled.
13:32:13,485 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-9, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Overriding the default retries config to the recommended value of 2147483647 since the idempotent producer is enabled.
13:32:13,485 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,485 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-13, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] ProducerId set to -1 with epoch -1
13:32:13,485 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,485 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-9, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Overriding the default acks to all since idempotence is enabled.
13:32:13,486 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-12, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] ProducerId set to -1 with epoch -1
13:32:13,488 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,489 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,490 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,490 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,490 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-11, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] ProducerId set to -1 with epoch -1
13:32:13,490 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-8, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] ProducerId set to -1 with epoch -1
13:32:13,492 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka version: 2.2.0
13:32:13,492 INFO  org.apache.kafka.common.utils.AppInfoParser                   - Kafka commitId: 05fcfde8f69b0349
13:32:13,493 INFO  org.apache.kafka.clients.producer.internals.TransactionManager  - [Producer clientId=producer-9, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] ProducerId set to -1 with epoch -1
13:32:13,709 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,709 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,719 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,755 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,756 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,766 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:13,772 INFO  org.apache.kafka.clients.Metadata                             - Cluster ID: kh-ujucVTXyDe9WnWHC-7g
13:32:25,408 INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator     - Triggering checkpoint 2 @ 1605184345407 for job ca9c3d38f9f1f64902dccf832dd51c79.
13:33:13,467 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-10, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-5] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,478 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-14, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-9] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,486 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-13, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-10] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,487 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-12, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-11] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,490 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-11, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-19] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,491 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-8, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-15] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,494 INFO  org.apache.kafka.clients.producer.KafkaProducer               - [Producer clientId=producer-9, transactionalId=Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka-a15b4dd4812495cebdc94e33125ef858-1] Closing the Kafka producer with timeoutMillis = 9223372036854775807 ms.
13:33:13,554 INFO  com.ebs.flink.transferprocessor.utils.DateFilter              - Shutting down
13:33:13,554 INFO  org.apache.flink.runtime.taskmanager.Task                     - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from RUNNING to FAILED.
org.apache.kafka.common.errors.TimeoutException: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
Caused by: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
13:33:13,555 INFO  org.apache.flink.runtime.taskmanager.Task                     - Freeing task resources for Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9).
13:33:13,555 INFO  org.apache.flink.runtime.taskmanager.Task                     - Ensuring all FileSystem streams are closed for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) [FAILED]
13:33:13,556 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Un-registering task and sending final execution state FAILED to JobManager for task Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) a26c57a7c399abe1e83b9893a0197ff9.
13:33:13,558 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source: Read player events from Kafka -> Map Json to HashMap -> Add CanonicalTime as timestamp -> Filter dates not needed for backfill -> Sink: Post events to playerEvents Kafka (1/1) (a26c57a7c399abe1e83b9893a0197ff9) switched from RUNNING to FAILED.
org.apache.kafka.common.errors.TimeoutException: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
Caused by: org.apache.kafka.common.errors.TimeoutException: Timeout expired while initializing transactional state in 60000ms.
13:33:13,559 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - Calculating tasks to restart to recover the failed task fce6063a4f1b368072cffb5f5909e445_0.
13:33:13,560 INFO  org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy  - 1 tasks should be restarted to recover the failed task fce6063a4f1b368072cffb5f5909e445_0. 
13:33:13,560 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Transfer Processor (ca9c3d38f9f1f64902dccf832dd51c79) switched from state RUNNING to RESTARTING.

Reply via email to