Re: Resource changed on src filesystem after upgrade

2021-01-29 Thread Mark Davis
Hi Xintong Song, > - Does this error happen for every of your dataset jobs? For a problematic > job, does it happen for every container? > - What is the `jobs.jar`? Is it under `lib/`, `opt` of your client side > filesystem, or specified as `yarn.ship-files`, `yarn.ship-archives` or > `yarn.pro

Resource changed on src filesystem after upgrade

2021-01-17 Thread Mark Davis
Hi all, I am upgrading my DataSet jobs from Flink 1.8 to 1.12. After the upgrade I started to receive the errors like this one: 14:12:57,441 INFO org.apache.flink.runtime.resourcemanager.active.ActiveResourceManager - Worker container_e120_1608377880203_0751_01_000112 is terminated. Diagnostics:

Re: Resource leak in DataSourceNode?

2020-08-30 Thread Mark Davis
Best, > Robert > > On Wed, Aug 26, 2020 at 4:05 PM Mark Davis wrote: > >> Hi, >> >> I am trying to investigate a problem with non-released resources in my >> application. >> >> I have a stateful application which submits Flink DataSetjobs using c

Resource leak in DataSourceNode?

2020-08-26 Thread Mark Davis
Hi, I am trying to investigate a problem with non-released resources in my application. I have a stateful application which submits Flink DataSetjobs using code very similar to the code in CliFrontend. I noticed what I am getting a lot of non-closed connections to my data store (HBase in my ca

Re: Run command after Batch is finished

2020-06-09 Thread Mark Davis
>> I also cc Aljoscha, maybe, he has more ideas. >> >> Best, >> Andrey >> >> On Sun, Jun 7, 2020 at 1:35 PM Mark Davis wrote: >> >>> Hi Jeff, >>> >>> Unfortunately this is not good enough for me. >>> My clients are very volatile

Re: Run command after Batch is finished

2020-06-07 Thread Mark Davis
the client side where ExecutionEnvironment is created. > > Mark Davis 于2020年6月6日周六 下午8:14写道: > >> Hi Jeff, >> >> Thank you very much! That is exactly what I need. >> >> Where the listener code will run in the cluster deployment(YARN, k8s)? >> Will it be sent

Re: Run command after Batch is finished

2020-06-06 Thread Mark Davis
ner which you can register to ExecutionEnvironment. > > https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/core/execution/JobListener.java > > Mark Davis 于2020年6月6日周六 上午12:00写道: > >> Hi there, >> >> I am running a Batch job with s

Run command after Batch is finished

2020-06-05 Thread Mark Davis
Hi there, I am running a Batch job with several outputs. Is there a way to run some code(e.g. release a distributed lock) after all outputs are finished? Currently I do this in a try-finally block around ExecutionEnvironment.execute() call, but I have to switch to the detached execution mode -

Re: DataSet API: HBase ScannerTimeoutException and double Result processing

2019-11-25 Thread Mark Davis
Hi Flavio, >> When the resultScanner dies because of a timeout (this happens a lot when >> you have backpressure and the time between 2 consecutive reads exceed the >> scanner timeout), the code creates a new scanner and restart from where it >> was (starRow = currentRow). >> So there should no

DataSet API: HBase ScannerTimeoutException and double Result processing

2019-11-23 Thread Mark Davis
Hello, I am reading Results from an HBase table and process them with Batch API. Everything works fine until I receive a ScannerTimeoutException from HBase. Maybe my transformations get stuck or a GC pause happen - hard to tell. The HBase Client restarts the scan and the processing continues. Ex