Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1B36A200D39 for ; Sat, 11 Nov 2017 23:02:48 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B1476160C03; Sat, 11 Nov 2017 22:02:37 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id DA4E41609E5 for ; Sat, 11 Nov 2017 23:02:35 +0100 (CET) Received: (qmail 4146 invoked by uid 500); 11 Nov 2017 22:02:34 -0000 Mailing-List: contact user-help@gobblin.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@gobblin.incubator.apache.org Delivered-To: mailing list user@gobblin.incubator.apache.org Received: (qmail 4136 invoked by uid 99); 11 Nov 2017 22:02:34 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 11 Nov 2017 22:02:34 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 99C58DA8A2 for ; Sat, 11 Nov 2017 22:02:33 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -0.152 X-Spam-Level: X-Spam-Status: No, score=-0.152 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, KAM_LOTSOFHASH=0.25, RCVD_IN_DNSWL_MED=-2.3, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (1024-bit key) header.d=linkedin.com header.b=SVwEvskr; dkim=pass (1024-bit key) header.d=microsoft.onmicrosoft.com header.b=Hpz2/nGs Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id MYYzdrOORyXJ for ; Sat, 11 Nov 2017 22:02:30 +0000 (UTC) Received: from mail723.prod.linkedin.com (mail723.linkedin.com [108.174.0.123]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 4C5F85F3BB for ; Sat, 11 Nov 2017 22:02:30 +0000 (UTC) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=linkedin.com; s=proddkim1024; t=1510437742; bh=L0KxsxyQd6skCWOnkBw/tckXmEzYOSDNc0BOBpr+edw=; h=From:To:Subject:Date:Content-Type:MIME-Version; b=SVwEvskryd0bSyGAlE52qJqvN5zeeE0hsPKrk4k9ZswnT5FQRjEUDYZ391OxMKhAn HzwJWD6fJ1YFyxhIlWiu5BoJajZDdxYfjZA4UFlpJZvZueEqVHwEdvNilugs1RU+tj syZjAFsUheMk5se05/T1FxtX2MQCw7eTqDCUm90M= Authentication-Results: mail723.prod.linkedin.com x-tls.subject="/C=US/ST=WA/L=Redmond/O=Microsoft Corporation/OU=Microsoft Corporation/CN=mail.protection.outlook.com"; auth=pass (cipher=ECDHE-RSA-AES256-SHA384) Authentication-Results: mail723.prod.linkedin.com; iprev=pass policy.iprev="2a01:111:f400:7e4a::206"; spf=softfail smtp.mailfrom="hutran@linkedin.com" smtp.helo="nam03-by2-obe.outbound.protection.outlook.com"; dkim=pass header.d=microsoft.onmicrosoft.com; tls=pass (verified) key.ciphersuite="TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384" key.length="256" tls.v="tlsv1.2" cert.client="C=US,ST=WA,L=Redmond,O=Microsoft Corporation,OU=Microsoft Corporation,CN=mail.protection.outlook.com" cert.clientissuer="C=US,ST=Washington,L=Redmond,O=Microsoft Corporation,OU=Microsoft IT,CN=Microsoft IT SSL SHA2" Received: from [2a01:111:f400:7e4a::206] ([2a01:111:f400:7e4a::206.1074] helo=NAM03-BY2-obe.outbound.protection.outlook.com) by mail723.prod.linkedin.com (envelope-from ) (ecelerity 4.2.1.55028 r(Core:4.2.1.12)) with ESMTPS (cipher=ECDHE-RSA-AES256-SHA384 subject="/C=US/ST=WA/L=Redmond/O=Microsoft Corporation/OU=Microsoft Corporation/CN=mail.protection.outlook.com") id 0A/C1-04533-E63770A5; Sat, 11 Nov 2017 22:02:22 +0000 DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=microsoft.onmicrosoft.com; s=selector1-microsoft-onmicrosoft-com; h=From:Date:Subject:Message-ID:Content-Type:MIME-Version; bh=L0KxsxyQd6skCWOnkBw/tckXmEzYOSDNc0BOBpr+edw=; b=Hpz2/nGsyLhmQaE/q2KZtAfeQEFoy4vC/6+AQGhxB6IwcqfxWIV4ORSUhghT4OO1/oqIO09rKlbjCqZU+F1ZqnXaWrxhsPH1oSG3TCGirt/3VYYh4jnrkv8WUwD/FM4EGBZYBxDezcYe++oGnqH6ezcv3x63N6zATznmN4EqAmE= Received: from DM5PR2101MB0805.namprd21.prod.outlook.com (10.167.105.149) by DM5PR2101MB0805.namprd21.prod.outlook.com (10.167.105.149) with Microsoft SMTP Server (version=TLS1_2, cipher=TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384_P384) id 15.20.218.2; Sat, 11 Nov 2017 22:02:19 +0000 Received: from DM5PR2101MB0805.namprd21.prod.outlook.com ([fe80::e480:dc0:b3c2:1f51]) by DM5PR2101MB0805.namprd21.prod.outlook.com ([fe80::e480:dc0:b3c2:1f51%13]) with mapi id 15.20.0218.004; Sat, 11 Nov 2017 22:02:19 +0000 From: Hung Tran To: "user@gobblin.incubator.apache.org" , "dev@gobblin.incubator.apache.org" Subject: Re: Corrupted state file when Jobs are being run in parallel. Thread-Topic: Corrupted state file when Jobs are being run in parallel. Thread-Index: AQHTWvVH7nDFW0zd3Eum7kNDOrpRd6MPugfR Date: Sat, 11 Nov 2017 22:02:19 +0000 Message-ID: References: In-Reply-To: Accept-Language: en-US Content-Language: en-US X-MS-Has-Attach: X-MS-TNEF-Correlator: x-originating-ip: [108.228.14.139] x-ms-publictraffictype: Email x-microsoft-exchange-diagnostics: 1;DM5PR2101MB0805;6:R9AdnNVIIq/5f5o6E6LjdDZJpnDkzTaN3r110zkBgn48N1Z8k/ARBh2CxVkh8xqMCyYXTc/NOySD82NpGWltBjo1shxRD6IHygGVnrTJo8M5YaU1eVcBpvdtoRmzqOQDQjxpSg+yuwHn0dyyDa8ITlOdWnClf7Yaj8YEhRcKiFt/T0n3Jjq9E7RqL2Op18D3aKTEqLkK/3rN03e3EeaOcGv5IX1jg3d3f3laSgpODrJms3u0eWK/dsIGxFWm+hN57meXxULzYxFqduvbQpyYikrCT08eI7598Gek+2wkdK2wRQ4bNTwEmM+KCcGdGGR6KmHRlvVDnE3jusBQL818j15yM7IKOfhpDNPJ7QZvdow=;5:Ntp9MxHOURsg1oICNV7ONcttDn9Ey8yIt6czhb1j73qhYuNti/kvDmMqRPXlsj8FBiVr3kYYAn/rZD/PUZqroOMvmlBMi22BSi5YuJeWyGMVPbA23/RYy9yQh2rYMipbwMIle6V8CBuZF6OKrdc5QtEibSGNhxelgr587YbkpoE=;24:eIO3CLxWR2mZhuGMSlIWb42zuqIHfiQTIOu8s5uxLxNcxm+p14R71Nw2NF0KMo39VDSt2wy7dgE+PotKU7lkSBKq/chotxszfO0eTvkWgvU=;7:OTny2ATcqm+azYtYabhGA17pUdiP304CBuK+rJOSlysXDbUU40Yi6HTORvUu2Bi/6SLrjCAhTbxrwIEPl2MYRRAnfUkArdZWQEUt5M/PTQ5iiYYF0t9dkB43DBAAIc5Dhz6TPB9xI9h9dcjum2MiFw5T0DGMiisBzf1lTi35+2EpL41l7vKxtGZBmxx8DuqGI+XIL9e1P+jEf5nv4ZZ9btOlVGEkvi/ygMxP/0x0CXHhoIzZZZEdMavfOw4UoACr x-ms-exchange-antispam-srfa-diagnostics: SSOS; x-ms-office365-filtering-correlation-id: 661d9f87-3729-42d6-427e-08d5294fe0cc x-ms-office365-filtering-ht: Tenant x-microsoft-antispam: UriScan:;BCL:0;PCL:0;RULEID:(22001)(4534020)(4602075)(4627115)(201703031133081)(201702281549075)(48565401081)(2017052603258);SRVR:DM5PR2101MB0805; x-ms-traffictypediagnostic: DM5PR2101MB0805: authentication-results: spf=none (sender IP is ) smtp.mailfrom=hutran@linkedin.com; x-o365ent-eop-header: Message Processed By - CBR_LInkedIn_Mail_To_External x-microsoft-antispam-prvs: x-exchange-antispam-report-test: UriScan:; x-exchange-antispam-report-cfa-test: BCL:0;PCL:0;RULEID:(100000700101)(100105000095)(100000701101)(100105300095)(100000702101)(100105100095)(61425038)(6040450)(2401047)(8121501046)(5005006)(93006095)(93001095)(3231022)(3002001)(100000703101)(100105400095)(10201501046)(6055026)(61426038)(61427038)(6041248)(201703131423075)(201702281528075)(201703061421075)(201703061406153)(20161123560025)(20161123558100)(20161123562025)(20161123564025)(20161123555025)(6072148)(201708071742011)(100000704101)(100105200095)(100000705101)(100105500095);SRVR:DM5PR2101MB0805;BCL:0;PCL:0;RULEID:(100000800101)(100110000095)(100000801101)(100110300095)(100000802101)(100110100095)(100000803101)(100110400095)(100000804101)(100110200095)(100000805101)(100110500095);SRVR:DM5PR2101MB0805; x-forefront-prvs: 0488C54DB4 x-forefront-antispam-report: SFV:NSPM;SFS:(10019020)(376002)(39860400002)(346002)(199003)(47662002)(189002)(377424004)(316002)(66066001)(110136005)(3280700002)(3660700001)(2906002)(6606003)(53546010)(74316002)(5660300001)(7736002)(478600001)(450100002)(25786009)(966005)(229853002)(97736004)(14454004)(99286004)(4001150100001)(8990500004)(2950100002)(7696004)(50986999)(101416001)(6436002)(6246003)(68736007)(79102999)(105586002)(5250100002)(2900100001)(54896002)(19627405001)(106356001)(55016002)(9686003)(189998001)(8676002)(8936002)(236005)(2501003)(33656002)(86362001)(53936002)(606006)(81166006)(6506006)(81156014)(575784001)(102836003)(3846002)(6116002)(6306002)(10090500001)(76176999)(54356999);DIR:OUT;SFP:1102;SCL:1;SRVR:DM5PR2101MB0805;H:DM5PR2101MB0805.namprd21.prod.outlook.com;FPR:;SPF:None;PTR:InfoNoRecords;A:1;MX:1;LANG:en; received-spf: None (protection.outlook.com: linkedin.com does not designate permitted sender hosts) spamdiagnosticoutput: 1:99 spamdiagnosticmetadata: NSPM Content-Type: multipart/alternative; boundary="_000_DM5PR2101MB08054452805A80E27D3AA081A7550DM5PR2101MB0805_" MIME-Version: 1.0 X-OriginatorOrg: linkedin.com X-MS-Exchange-CrossTenant-Network-Message-Id: 661d9f87-3729-42d6-427e-08d5294fe0cc X-MS-Exchange-CrossTenant-originalarrivaltime: 11 Nov 2017 22:02:19.1868 (UTC) X-MS-Exchange-CrossTenant-fromentityheader: Hosted X-MS-Exchange-CrossTenant-id: 72f988bf-86f1-41af-91ab-2d7cd011db47 X-MS-Exchange-Transport-CrossTenantHeadersStamped: DM5PR2101MB0805 archived-at: Sat, 11 Nov 2017 22:02:48 -0000 --_000_DM5PR2101MB08054452805A80E27D3AA081A7550DM5PR2101MB0805_ Content-Type: text/plain; charset="us-ascii" Content-Transfer-Encoding: quoted-printable Hi Vicky, What's the underlying file system you are using? Have you checked to make sure there are not multiple instances of the job r= unning concurrently or make use of the job lock? Also, the state store and state are separate things. You can still pass sta= te from the source to the extractor without the state store enabled. The st= ate store is used to transfer state across executions, like cases where a w= atermark is used to resume an incremental pull. Hung. ________________________________ From: Vicky Kak Sent: Saturday, November 11, 2017 5:58:59 AM To: user@gobblin.incubator.apache.org; dev@gobblin.incubator.apache.org Subject: Corrupted state file when Jobs are being run in parallel. Hi Guys, I have been running the stress tests and am seeing the following errors Error 1 ***************************************************************************= ****************************************************** 017-11-11 11:20:56 UTC INFO [pool-11-thread-421] org.apache.hadoop.fs.FSInputChecker 284 - Found checksum error: b[0, 512]=3D53455106196f72672e6170616368652e6861646f6f702e696f2e5465787425676f62= 626c696e2e72756e74696d652e4a6f6253746174652444617461736574537461746501012a6= f72672e6170616368652e6861646f6f702e696f2e636f6d70726573732e44656661756c7443= 6f6465630000000044e218b9e6aad3f1aa97f2210fb5c7f0ffffffff44e218b9e6aad3f1aa9= 7f2210fb5c7f00109789c6304000002000209789c630000000100010b789cebb3d502000251= 00f68e0ab4789ced5b7b73dbc611971c3b8d5ff233b6d324ad861337e9d804013e445169264= 3d1924c51a26489962da71ece013890270238f870904479fc1592ffdb4fd1e9f4b364a6dfa3= ff770f0fbe244384eca6c998d2f081bbddc5deede2f6777bcbcf974da275d8266ae1a543ce9= 0c629dbf44cb3a9e48ae93daa3663fa9b4a419195ac5c2a147373a5a9a9e9e6df4adf3c0a3e= e7ff39e5ff5da7172b1bebebd54663097aa6a6c52b995c9923b7333e79530e15f93954e41f8= 122d7fe0d6f8f6fe0805bb291855d0769f8aee14b961c102da17d4625576b630b5d7ae561d6= 954c64b7ce75d81742098639b4f036c348772835250b1dbae4084f672fba1c1a2d89e85f159= 031870d944fe7545d4be70b46313d5f9071ba24e772459445322aea331479bc2df96f1e33bf= 6d73eeb80b998c4d74506c1f3349a356c627ca4a72467c520637fa9e org.apache.hadoop.fs.ChecksumException: Checksum error: file:/home/Installable/gobblin-dist/working-dir/state-store/FlickrPageExtra= ctorPull_137/current.jst at 0 exp: 36820587 got: 91149211 at org.apache.hadoop.fs.FSInputChecker.verifySums(FSInputChecker.java:322) at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:2= 78) at org.apache.hadoop.fs.FSInputChecker.fill(FSInputChecker.java:213= ) at org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:231) at org.apache.hadoop.fs.FSInputChecker.read(FSInputChecker.java:195= ) at java.io.DataInputStream.readFully(DataInputStream.java:195) at java.io.DataInputStream.readFully(DataInputStream.java:169) at org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1845) at org.apache.hadoop.io.SequenceFile$Reader.initialize(SequenceFile.java:1810) at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1759) at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1773) at gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateStore.java:119) at gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesByUrns(FsDatasetS= tateStore.java:173) at gobblin.runtime.JobContext.(JobContext.java:136) at gobblin.runtime.AbstractJobLauncher.(AbstractJobLauncher.java:131) at gobblin.runtime.local.LocalJobLauncher.(LocalJobLauncher.java:62) at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:8= 0) at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:5= 9) at com.bph.JobLauncherResource.search(JobLauncherResource.java:107) at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImp= l.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at com.linkedin.restli.internal.server.RestLiMethodInvoker.doInvoke(RestLiMeth= odInvoker.java:186) at com.linkedin.restli.internal.server.RestLiMethodInvoker.invoke(RestLiMethod= Invoker.java:141) at com.linkedin.restli.server.RestLiServer.handleResourceRequest(RestLiServer.= java:286) at com.linkedin.restli.server.RestLiServer.doHandleRequest(RestLiServer.java:1= 67) at com.linkedin.restli.server.BaseRestServer.handleRequest(BaseRestServer.java= :56) at com.linkedin.restli.server.DelegatingTransportDispatcher.handleRestRequest(= DelegatingTransportDispatcher.java:56) at com.linkedin.r2.filter.transport.DispatcherRequestFilter.onRestRequest(Disp= atcherRequestFilter.java:81) at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.compression.ServerCompressionFilter.onRestRequest(Se= rverCompressionFilter.java:126) at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:59) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterChainImpl.java:1= 03) at com.linkedin.r2.filter.transport.FilterChainDispatcher.handleRestRequest(Fi= lterChainDispatcher.java:74) at com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:95) at com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:62) at com.linkedin.r2.transport.http.server.HttpNettyServer$Handler.messageReceiv= ed(HttpNettyServer.java:171) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleC= hannelUpstreamHandler.java:80) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelP= ipeline.java:545) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext= .sendUpstream(DefaultChannelPipeline.java:754) at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunn= able.java:69) at org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor$Chil= dExecutor.run(OrderedMemoryAwareThreadPoolExecutor.java:316) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617) at java.lang.Thread.run(Thread.java:745) 2017-11-11 11:20:56 UTC ERROR [pool-11-thread-421] com.bph.JobLauncherResource 110 - Job Id fk_137 failed while searching key beryls Failed to create job launcher: org.apache.hadoop.fs.ChecksumException: Checksum error: file:/home/Installable/gobblin-dist/working-dir/state-store/FlickrPageExtra= ctorPull_137/current.jst at 0 exp: 36820587 got: 91149211 2017-11-11 11:20:56 UTC INFO [pool-11-thread-402] gobblin.util.ExecutorsUtils 125 - Attempting to shutdown ExecutorService: java.util.concurrent.ThreadPoolExecutor@6bce96a5[Shutting down, pool size = =3D 1, active threads =3D 0, queued tasks =3D 0, completed tasks =3D 1] 2017-11-11 11:20:56 UTC INFO [pool-11-thread-402] gobblin.util.ExecutorsUtils 144 - Successfully shutdown ExecutorService: java.util.concurrent.ThreadPoolExecutor@6bce96a5[Terminated, pool size =3D = 0, active threads =3D 0, queued tasks =3D 0, completed tasks =3D 1] ***************************************************************************= ****************************************************** Error 2: ***************************************************************************= ****************************************************** 2017-11-10 10:24:10 UTC WARN [pool-11-thread-13] org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker 154 - Problem opening checksum file: file:/home/Installable/gobblin-dist/working-dir/state-store/YoutubePageExtr= actorPull_138/current.jst. Ignoring exception: java.io.EOFException at java.io.DataInputStream.readFully(DataInputStream.java:197) at java.io.DataInputStream.readFully(DataInputStream.java:169) at org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.(Check= sumFileSystem.java:146) at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:339) at org.apache.hadoop.io.SequenceFile$Reader.openFile(SequenceFile.java:1832) at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1752) at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1773) at gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateStore.java:119) at gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesByUrns(FsDatasetS= tateStore.java:173) at gobblin.runtime.JobContext.(JobContext.java:136) at gobblin.runtime.AbstractJobLauncher.(AbstractJobLauncher.java:131) at gobblin.runtime.local.LocalJobLauncher.(LocalJobLauncher.java:62) at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:8= 0) at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:5= 9) at com.bph.JobLauncherResource.search(JobLauncherResource.java:107) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:6= 2) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImp= l.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at com.linkedin.restli.internal.server.RestLiMethodInvoker.doInvoke(RestLiMeth= odInvoker.java:186) at com.linkedin.restli.internal.server.RestLiMethodInvoker.invoke(RestLiMethod= Invoker.java:141) at com.linkedin.restli.server.RestLiServer.handleResourceRequest(RestLiServer.= java:286) at com.linkedin.restli.server.RestLiServer.doHandleRequest(RestLiServer.java:1= 67) at com.linkedin.restli.server.BaseRestServer.handleRequest(BaseRestServer.java= :56) at com.linkedin.restli.server.DelegatingTransportDispatcher.handleRestRequest(= DelegatingTransportDispatcher.java:56) at com.linkedin.r2.filter.transport.DispatcherRequestFilter.onRestRequest(Disp= atcherRequestFilter.java:81) at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.compression.ServerCompressionFilter.onRestRequest(Se= rverCompressionFilter.java:126) at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:59) at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50) at com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterChainImpl.java:1= 03) at com.linkedin.r2.filter.transport.FilterChainDispatcher.handleRestRequest(Fi= lterChainDispatcher.java:74) at com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:95) at com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:62) at com.linkedin.r2.transport.http.server.HttpNettyServer$Handler.messageReceiv= ed(HttpNettyServer.java:171) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleC= hannelUpstreamHandler.java:80) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelP= ipeline.java:545) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext= .sendUpstream(DefaultChannelPipeline.java:754) at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunn= able.java:69) at org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor$Chil= dExecutor.run(OrderedMemoryAwareThreadPoolExecutor.java:316) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617) at java.lang.Thread.run(Thread.java:745) 2017-11-10 10:24:11 UTC ERROR [pool-11-thread-13] com.bph.JobLauncherResource 110 - Job Id yt_138 failed while searching key ostfold Failed to create job launcher: java.io.EOFException ***************************************************************************= ****************************************************** Error 3 ***************************************************************************= ****************************************************** 2017-11-10 13:38:49 UTC ERROR [Commit-thread-0] gobblin.runtime.SafeDatasetCommit 118 - Failed to persist dataset state for dataset of job job_TwitterPageExtractorPull_135_1510321111647 java.io.FileNotFoundException: Failed to rename /home/Installable/gobblin-dist/working-dir/state-store/TwitterPageExtractor= Pull_135/_tmp_/current.jst to /home/Installable/gobblin-dist/working-dir/state-store/TwitterPageExtractor= Pull_135/current.jst: src not found at gobblin.util.HadoopUtils.renamePath(HadoopUtils.java:173) at gobblin.util.HadoopUtils.renamePath(HadoopUtils.java:164) at gobblin.util.HadoopUtils.copyFile(HadoopUtils.java:333) at gobblin.metastore.FsStateStore.createAlias(FsStateStore.java:283) at gobblin.runtime.FsDatasetStateStore.persistDatasetState(FsDatasetStateStore= .java:221) at gobblin.runtime.SafeDatasetCommit.persistDatasetState(SafeDatasetCommit.jav= a:255) at gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:115) at gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:43) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617) at java.lang.Thread.run(Thread.java:745) ***************************************************************************= ****************************************************** There errors are seeing during the stress tests for the same Jobs. For our use case we can't afford the jobs to fail due to system issue like above. I did some basic investigation and could find the issue could be happening to to non atomic operations on the state file which is of extension .jst. It seems we could disable the statestore, I looked at the following code in gobblin.runtime.JobContext::createStateStore ***************************************************************************= ****************************************************** if (jobProps.containsKey(ConfigurationKeys.STATE_STORE_ENABLED) && !Boolean.parseBoolean(jobProps.getProperty(ConfigurationKeys.STATE_STORE_EN= ABLED))) { return new NoopDatasetStateStore(stateStoreFs, stateStoreRootDir); } else { return new FsDatasetStateStore(stateStoreFs, stateStoreRootDir); } ***************************************************************************= ****************************************************** It seems that by disabling the statestore we may get over this issue, but for our case the source implementation is passing the information to the Extractor via the WorkUnitStoreState. We don't want the Job Retry features and hence did disable it as explained here https://gobblin.readthedocs.io/en/latest/user-guide/Configuration-Propertie= s-Glossary/#retry-properties I was expecting the disabling happening by setting the follwing only workunit.retry.enabled=3Dfalse we have to set this also task.maxretries=3D0 As we don't rely on retries would it not be good to have a flag what will ignore the the following calls when we have have workunit.retry.enabled=3Dfalse 1) Reading the initial value from the store 2) Commit the final state to the store. As mentioned about we can't disable the state store as we are generating some data in the Source implementation and passed to the corresponding Extractor implementation via State. I do anticipate of having these issues in GAAS too. I will be working to fix this issue as this is a critical issue for us. Thanks, Vicky --_000_DM5PR2101MB08054452805A80E27D3AA081A7550DM5PR2101MB0805_ Content-Type: text/html; charset="us-ascii" Content-Transfer-Encoding: quoted-printable

Hi Vicky,


What's the underlying file system you are using?


Have you checked to make sure there are not multiple instances of the jo= b running concurrently or make use of the job lock?


Also, the state store and state are separate things. You can still pass = state from the source to the extractor without the state store enabled. The= state store is used to transfer state across executions, like cases where = a watermark is used to resume an incremental pull.


Hung.


From: Vicky Kak <vicky.k= ak@gmail.com>
Sent: Saturday, November 11, 2017 5:58:59 AM
To: user@gobblin.incubator.apache.org; dev@gobblin.incubator.apache.= org
Subject: Corrupted state file when Jobs are being run in parallel.
 
Hi Guys,

I have been running the stress tests and am seeing the following errors

Error 1
***************************************************************************= ******************************************************
017-11-11 11:20:56 UTC INFO  [pool-11-thread-421]
org.apache.hadoop.fs.FSInputChecker  284 - Found checksum error: b[0,<= br> 512]=3D53455106196f72672e6170616368652e6861646f6f702e696f2e5465787425676f62= 626c696e2e72756e74696d652e4a6f6253746174652444617461736574537461746501012a6= f72672e6170616368652e6861646f6f702e696f2e636f6d70726573732e44656661756c7443= 6f6465630000000044e218b9e6aad3f1aa97f2210fb5c7f0ffffffff44e218b9e6aad3f1aa9= 7f2210fb5c7f00109789c6304000002000209789c630000000100010b789cebb3d502000251= 00f68e0ab4789ced5b7b73dbc611971c3b8d5ff233b6d324ad861337e9d804013e445169264= 3d1924c51a26489962da71ece013890270238f870904479fc1592ffdb4fd1e9f4b364a6dfa3= ff770f0fbe244384eca6c998d2f081bbddc5deede2f6777bcbcf974da275d8266ae1a543ce9= 0c629dbf44cb3a9e48ae93daa3663fa9b4a419195ac5c2a147373a5a9a9e9e6df4adf3c0a3e= e7ff39e5ff5da7172b1bebebd54663097aa6a6c52b995c9923b7333e79530e15f93954e41f8= 122d7fe0d6f8f6fe0805bb291855d0769f8aee14b961c102da17d4625576b630b5d7ae561d6= 954c64b7ce75d81742098639b4f036c348772835250b1dbae4084f672fba1c1a2d89e85f159= 031870d944fe7545d4be70b46313d5f9071ba24e772459445322aea331479bc2df96f1e33bf= 6d73eeb80b998c4d74506c1f3349a356c627ca4a72467c520637fa9e
org.apache.hadoop.fs.ChecksumException: Checksum error:
file:/home/Installable/gobblin-dist/working-dir/state-store/FlickrPageExtra= ctorPull_137/current.jst
at 0 exp: 36820587 got: 91149211
        at
org.apache.hadoop.fs.FSInputChecker.verifySums(FSInputChecker.java:322)
        at
org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInputChecker.java:2= 78)
        at org.apache.hadoop.fs.FSInputC= hecker.fill(FSInputChecker.java:213)
        at
org.apache.hadoop.fs.FSInputChecker.read1(FSInputChecker.java:231)
        at org.apache.hadoop.fs.FSInputC= hecker.read(FSInputChecker.java:195)
        at java.io.DataInputStream.readF= ully(DataInputStream.java:195)
        at java.io.DataInputStream.readF= ully(DataInputStream.java:169)
        at
org.apache.hadoop.io.SequenceFile$Reader.init(SequenceFile.java:1845)
        at
org.apache.hadoop.io.SequenceFile$Reader.initialize(SequenceFile.java:1810)=
        at
org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:175= 9)
        at
org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:177= 3)
        at
gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateStore.java:119)         at
gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesByUrns(FsDatasetS= tateStore.java:173)
        at gobblin.runtime.JobContext.&l= t;init>(JobContext.java:136)
        at
gobblin.runtime.AbstractJobLauncher.<init>(AbstractJobLauncher.java:1= 31)
        at
gobblin.runtime.local.LocalJobLauncher.<init>(LocalJobLauncher.java:6= 2)
        at
gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:8= 0)
        at
gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:5= 9)
        at com.bph.JobLauncherResource.s= earch(JobLauncherResource.java:107)
        at sun.reflect.GeneratedMethodAc= cessor8.invoke(Unknown Source)
        at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImp= l.java:43)
        at java.lang.reflect.Method.invo= ke(Method.java:498)
        at
com.linkedin.restli.internal.server.RestLiMethodInvoker.doInvoke(RestLiMeth= odInvoker.java:186)
        at
com.linkedin.restli.internal.server.RestLiMethodInvoker.invoke(RestLiMethod= Invoker.java:141)
        at
com.linkedin.restli.server.RestLiServer.handleResourceRequest(RestLiServer.= java:286)
        at
com.linkedin.restli.server.RestLiServer.doHandleRequest(RestLiServer.java:1= 67)
        at
com.linkedin.restli.server.BaseRestServer.handleRequest(BaseRestServer.java= :56)
        at
com.linkedin.restli.server.DelegatingTransportDispatcher.handleRestRequest(= DelegatingTransportDispatcher.java:56)
        at
com.linkedin.r2.filter.transport.DispatcherRequestFilter.onRestRequest(Disp= atcherRequestFilter.java:81)
        at
com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.compression.ServerCompressionFilter.onRestRequest(Se= rverCompressionFilter.java:126)
        at
com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:59)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterChainImpl.java:1= 03)
        at
com.linkedin.r2.filter.transport.FilterChainDispatcher.handleRestRequest(Fi= lterChainDispatcher.java:74)
        at
com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:95)
        at
com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:62)
        at
com.linkedin.r2.transport.http.server.HttpNettyServer$Handler.messageReceiv= ed(HttpNettyServer.java:171)
        at
org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleC= hannelUpstreamHandler.java:80)
        at
org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelP= ipeline.java:545)
        at
org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext= .sendUpstream(DefaultChannelPipeline.java:754)
        at
org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunn= able.java:69)
        at
org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor$Chil= dExecutor.run(OrderedMemoryAwareThreadPoolExecutor.java:316)
        at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142)
        at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617)
        at java.lang.Thread.run(Thread.j= ava:745)
2017-11-11 11:20:56 UTC ERROR [pool-11-thread-421]
com.bph.JobLauncherResource  110 -  Job Id fk_137 failed while se= arching
key beryls Failed to create job launcher:
org.apache.hadoop.fs.ChecksumException: Checksum error:
file:/home/Installable/gobblin-dist/working-dir/state-store/FlickrPageExtra= ctorPull_137/current.jst
at 0 exp: 36820587 got: 91149211
2017-11-11 11:20:56 UTC INFO  [pool-11-thread-402]
gobblin.util.ExecutorsUtils  125 - Attempting to shutdown ExecutorServ= ice:
java.util.concurrent.ThreadPoolExecutor@6bce96a5[Shutting down, pool size = =3D
1, active threads =3D 0, queued tasks =3D 0, completed tasks =3D 1]
2017-11-11 11:20:56 UTC INFO  [pool-11-thread-402]
gobblin.util.ExecutorsUtils  144 - Successfully shutdown ExecutorServi= ce:
java.util.concurrent.ThreadPoolExecutor@6bce96a5[Terminated, pool size =3D = 0,
active threads =3D 0, queued tasks =3D 0, completed tasks =3D 1]

***************************************************************************= ******************************************************

Error 2:
***************************************************************************= ******************************************************

2017-11-10 10:24:10 UTC WARN  [pool-11-thread-13]
org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker  154 -<= br> Problem opening checksum file:
file:/home/Installable/gobblin-dist/working-dir/state-store/YoutubePageExtr= actorPull_138/current.jst.
Ignoring exception:
java.io.EOFException
        at java.io.DataInputStream.readF= ully(DataInputStream.java:197)
        at java.io.DataInputStream.readF= ully(DataInputStream.java:169)
        at
org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker.<init>= (ChecksumFileSystem.java:146)
        at
org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:339)         at
org.apache.hadoop.io.SequenceFile$Reader.openFile(SequenceFile.java:1832)         at
org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:175= 2)
        at
org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile.java:177= 3)
        at
gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateStore.java:119)         at
gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesByUrns(FsDatasetS= tateStore.java:173)
        at gobblin.runtime.JobContext.&l= t;init>(JobContext.java:136)
        at
gobblin.runtime.AbstractJobLauncher.<init>(AbstractJobLauncher.java:1= 31)
        at
gobblin.runtime.local.LocalJobLauncher.<init>(LocalJobLauncher.java:6= 2)
        at
gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:8= 0)
        at
gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLauncherFactory.java:5= 9)
        at com.bph.JobLauncherResource.s= earch(JobLauncherResource.java:107)
        at sun.reflect.NativeMethodAcces= sorImpl.invoke0(Native Method)
        at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:6= 2)
        at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImp= l.java:43)
        at java.lang.reflect.Method.invo= ke(Method.java:498)
        at
com.linkedin.restli.internal.server.RestLiMethodInvoker.doInvoke(RestLiMeth= odInvoker.java:186)
        at
com.linkedin.restli.internal.server.RestLiMethodInvoker.invoke(RestLiMethod= Invoker.java:141)
        at
com.linkedin.restli.server.RestLiServer.handleResourceRequest(RestLiServer.= java:286)
        at
com.linkedin.restli.server.RestLiServer.doHandleRequest(RestLiServer.java:1= 67)
        at
com.linkedin.restli.server.BaseRestServer.handleRequest(BaseRestServer.java= :56)
        at
com.linkedin.restli.server.DelegatingTransportDispatcher.handleRestRequest(= DelegatingTransportDispatcher.java:56)
        at
com.linkedin.r2.filter.transport.DispatcherRequestFilter.onRestRequest(Disp= atcherRequestFilter.java:81)
        at
com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.compression.ServerCompressionFilter.onRestRequest(Se= rverCompressionFilter.java:126)
        at
com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdapter.onRequest(F= ilterChainImpl.java:328)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:55)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilter.java:59)
        at
com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterChainIterator.ja= va:50)
        at
com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterChainImpl.java:1= 03)
        at
com.linkedin.r2.filter.transport.FilterChainDispatcher.handleRestRequest(Fi= lterChainDispatcher.java:74)
        at
com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:95)
        at
com.linkedin.r2.transport.http.server.HttpDispatcher.handleRequest(HttpDisp= atcher.java:62)
        at
com.linkedin.r2.transport.http.server.HttpNettyServer$Handler.messageReceiv= ed(HttpNettyServer.java:171)
        at
org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleC= hannelUpstreamHandler.java:80)
        at
org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelP= ipeline.java:545)
        at
org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext= .sendUpstream(DefaultChannelPipeline.java:754)
        at
org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunn= able.java:69)
        at
org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor$Chil= dExecutor.run(OrderedMemoryAwareThreadPoolExecutor.java:316)
        at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142)
        at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617)
        at java.lang.Thread.run(Thread.j= ava:745)
2017-11-10 10:24:11 UTC ERROR [pool-11-thread-13]
com.bph.JobLauncherResource  110 -  Job Id yt_138 failed while se= arching
key ostfold Failed to create job launcher: java.io.EOFException

***************************************************************************= ******************************************************

Error 3
***************************************************************************= ******************************************************
2017-11-10 13:38:49 UTC ERROR [Commit-thread-0]
gobblin.runtime.SafeDatasetCommit  118 - Failed to persist dataset sta= te
for dataset  of job job_TwitterPageExtractorPull_135_1510321111647
java.io.FileNotFoundException: Failed to rename
/home/Installable/gobblin-dist/working-dir/state-store/TwitterPageExtractor= Pull_135/_tmp_/current.jst
to
/home/Installable/gobblin-dist/working-dir/state-store/TwitterPageExtractor= Pull_135/current.jst:
src not found
at gobblin.util.HadoopUtils.renamePath(HadoopUtils.java:173)
at gobblin.util.HadoopUtils.renamePath(HadoopUtils.java:164)
at gobblin.util.HadoopUtils.copyFile(HadoopUtils.java:333)
at gobblin.metastore.FsStateStore.createAlias(FsStateStore.java:283)
at
gobblin.runtime.FsDatasetStateStore.persistDatasetState(FsDatasetStateStore= .java:221)
at
gobblin.runtime.SafeDatasetCommit.persistDatasetState(SafeDatasetCommit.jav= a:255)
at gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:115)
at gobblin.runtime.SafeDatasetCommit.call(SafeDatasetCommit.java:43)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)<= br> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1= 142)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:= 617)
at java.lang.Thread.run(Thread.java:745)
***************************************************************************= ******************************************************

There errors are seeing during the stress tests for the same Jobs. For our<= br> use case we can't afford the jobs to fail due to system issue like above. I=
did some basic investigation and could find the issue could be happening to=
to non atomic operations on the state file which is of extension .jst. It seems we could disable the statestore, I looked at the following code in gobblin.runtime.JobContext::createStateStore
***************************************************************************= ******************************************************
if (jobProps.containsKey(ConfigurationKeys.STATE_STORE_ENABLED) &&<= br>
!Boolean.parseBoolean(jobProps.getProperty(ConfigurationKeys.STATE_STORE_EN= ABLED)))
{
      return new NoopDatasetStateStore(stateStoreF= s, stateStoreRootDir);
    } else {
      return new FsDatasetStateStore(stateStoreFs,= stateStoreRootDir);
    }
***************************************************************************= ******************************************************

It seems that by disabling the statestore we may get over this issue, but for our case the source implementation is passing the information to the Extractor via the WorkUnitStoreState.


We don't want the Job Retry features and hence did disable it as explained<= br> here
https://gobblin.readthedocs.io/en/= latest/user-guide/Configuration-Properties-Glossary/#retry-properties
I was expecting the disabling happening by setting the follwing only
workunit.retry.enabled=3Dfalse
we have to set this also
task.maxretries=3D0
As we don't rely on retries would it not be good to have a flag what will ignore the the following calls when we have have
workunit.retry.enabled=3Dfalse

1) Reading the initial value from the store
2) Commit the final state to the store.

As mentioned about we can't disable the state store as we are generating some data in the Source implementation and passed to the corresponding
Extractor implementation via State.

I do anticipate of having these issues in GAAS too.

I will be working to fix this issue as this is a critical issue for us.

Thanks,
Vicky
--_000_DM5PR2101MB08054452805A80E27D3AA081A7550DM5PR2101MB0805_--