gobblin-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Vicky Kak <vicky....@gmail.com>
Subject Re: Corrupted state file when Jobs are being run in parallel.
Date Sun, 12 Nov 2017 06:57:57 GMT
Hi Mohan,

I am not sure how you are using the Gobblin. You can follow the email chain
and find out how we are using.
You can enable debug logging via the log4j properties, I did some sample
application sometime back where I had tried it
https://github.com/dallaybatta/gobblin-examples/blob/master/src/main/resources/log4j.properties

You may read this
https://lists.apache.org/thread.html/3f73f7c32e6e92a9533bded39b51e22a03c513d86ae8813ba7097a12@%3Cuser.gobblin.apache.org%3E

Please post your queries in the separate post, having unrelated question in
the same thread will create confusion and will not be useful for referring
later too.

Thanks,
Vicky

On Sat, Nov 11, 2017 at 7:39 PM, Mohan <mohandoss.tr@gmail.com> wrote:

> Could you please tell me how to perform  stress test on Gobblin
> And please let me know how to enable debug option in log file. Thanks
>
> On Nov 11, 2017 7:29 PM, "Vicky Kak" <vicky.kak@gmail.com> wrote:
>
>> 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]=53455106196f72672e6170616368652e6861646f6f702e696f2e546
>> 5787425676f62626c696e2e72756e74696d652e4a6f62537461746524446
>> 17461736574537461746501012a6f72672e6170616368652e6861646f6f7
>> 02e696f2e636f6d70726573732e44656661756c74436f646563000000004
>> 4e218b9e6aad3f1aa97f2210fb5c7f0ffffffff44e218b9e6aad3f1aa97f
>> 2210fb5c7f00109789c6304000002000209789c630000000100010b789ce
>> bb3d50200025100f68e0ab4789ced5b7b73dbc611971c3b8d5ff233b6d32
>> 4ad861337e9d804013e4451692643d1924c51a26489962da71ece0138902
>> 70238f870904479fc1592ffdb4fd1e9f4b364a6dfa3ff770f0fbe244384e
>> ca6c998d2f081bbddc5deede2f6777bcbcf974da275d8266ae1a543ce90c
>> 629dbf44cb3a9e48ae93daa3663fa9b4a419195ac5c2a147373a5a9a9e9e
>> 6df4adf3c0a3ee7ff39e5ff5da7172b1bebebd54663097aa6a6c52b995c9
>> 923b7333e79530e15f93954e41f8122d7fe0d6f8f6fe0805bb291855d076
>> 9f8aee14b961c102da17d4625576b630b5d7ae561d6954c64b7ce75d8174
>> 2098639b4f036c348772835250b1dbae4084f672fba1c1a2d89e85f15903
>> 1870d944fe7545d4be70b46313d5f9071ba24e772459445322aea331479b
>> c2df96f1e33bf6d73eeb80b998c4d74506c1f3349a356c627ca4a72467c520637fa9e
>> org.apache.hadoop.fs.ChecksumException: Checksum error:
>> file:/home/Installable/gobblin-dist/working-dir/state-store/
>> FlickrPageExtractorPull_137/current.jst at 0 exp: 36820587 got: 91149211
>>         at org.apache.hadoop.fs.FSInputChecker.verifySums(FSInputChecke
>> r.java:322)
>>         at org.apache.hadoop.fs.FSInputChecker.readChecksumChunk(FSInpu
>> tChecker.java:278)
>>         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.<init>(SequenceFile
>> .java:1759)
>>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile
>> .java:1773)
>>         at gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateSto
>> re.java:119)
>>         at gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesBy
>> Urns(FsDatasetStateStore.java:173)
>>         at gobblin.runtime.JobContext.<init>(JobContext.java:136)
>>         at gobblin.runtime.AbstractJobLauncher.<init>(AbstractJobLaunch
>> er.java:131)
>>         at gobblin.runtime.local.LocalJobLauncher.<init>(LocalJobLaunch
>> er.java:62)
>>         at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLaunche
>> rFactory.java:80)
>>         at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLaunche
>> rFactory.java:59)
>>         at com.bph.JobLauncherResource.search(JobLauncherResource.java:
>> 107)
>>         at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
>>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMe
>> thodAccessorImpl.java:43)
>>         at java.lang.reflect.Method.invoke(Method.java:498)
>>         at com.linkedin.restli.internal.server.RestLiMethodInvoker.doIn
>> voke(RestLiMethodInvoker.java:186)
>>         at com.linkedin.restli.internal.server.RestLiMethodInvoker.invo
>> ke(RestLiMethodInvoker.java:141)
>>         at com.linkedin.restli.server.RestLiServer.handleResourceReques
>> t(RestLiServer.java:286)
>>         at com.linkedin.restli.server.RestLiServer.doHandleRequest(Rest
>> LiServer.java:167)
>>         at com.linkedin.restli.server.BaseRestServer.handleRequest(Base
>> RestServer.java:56)
>>         at com.linkedin.restli.server.DelegatingTransportDispatcher.han
>> dleRestRequest(DelegatingTransportDispatcher.java:56)
>>         at com.linkedin.r2.filter.transport.DispatcherRequestFilter.onR
>> estRequest(DispatcherRequestFilter.java:81)
>>         at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdap
>> ter.onRequest(FilterChainImpl.java:328)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:55)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.compression.ServerCompressionFilter.o
>> nRestRequest(ServerCompressionFilter.java:126)
>>         at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdap
>> ter.onRequest(FilterChainImpl.java:328)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:55)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:59)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterC
>> hainImpl.java:103)
>>         at com.linkedin.r2.filter.transport.FilterChainDispatcher.handl
>> eRestRequest(FilterChainDispatcher.java:74)
>>         at com.linkedin.r2.transport.http.server.HttpDispatcher.handleR
>> equest(HttpDispatcher.java:95)
>>         at com.linkedin.r2.transport.http.server.HttpDispatcher.handleR
>> equest(HttpDispatcher.java:62)
>>         at com.linkedin.r2.transport.http.server.HttpNettyServer$Handle
>> r.messageReceived(HttpNettyServer.java:171)
>>         at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleU
>> pstream(SimpleChannelUpstreamHandler.java:80)
>>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(
>> DefaultChannelPipeline.java:545)
>>         at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChanne
>> lHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
>>         at org.jboss.netty.handler.execution.ChannelEventRunnable.run(C
>> hannelEventRunnable.java:69)
>>         at org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPo
>> olExecutor$ChildExecutor.run(OrderedMemoryAwareThreadPoolExe
>> cutor.java:316)
>>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>> Executor.java:1142)
>>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.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/FlickrPageExtractorPull_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 = 1, active threads = 0, queued tasks = 0, completed tasks = 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 =
>> 0, active threads = 0, queued tasks = 0, completed tasks = 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/YoutubePageExtractorPull_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$ChecksumFSInputCheck
>> er.<init>(ChecksumFileSystem.java:146)
>>         at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSys
>> tem.java:339)
>>         at org.apache.hadoop.io.SequenceFile$Reader.openFile(SequenceFi
>> le.java:1832)
>>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile
>> .java:1752)
>>         at org.apache.hadoop.io.SequenceFile$Reader.<init>(SequenceFile
>> .java:1773)
>>         at gobblin.runtime.FsDatasetStateStore.getAll(FsDatasetStateSto
>> re.java:119)
>>         at gobblin.runtime.FsDatasetStateStore.getLatestDatasetStatesBy
>> Urns(FsDatasetStateStore.java:173)
>>         at gobblin.runtime.JobContext.<init>(JobContext.java:136)
>>         at gobblin.runtime.AbstractJobLauncher.<init>(AbstractJobLaunch
>> er.java:131)
>>         at gobblin.runtime.local.LocalJobLauncher.<init>(LocalJobLaunch
>> er.java:62)
>>         at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLaunche
>> rFactory.java:80)
>>         at gobblin.runtime.JobLauncherFactory.newJobLauncher(JobLaunche
>> rFactory.java:59)
>>         at com.bph.JobLauncherResource.search(JobLauncherResource.java:
>> 107)
>>         at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>         at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAcce
>> ssorImpl.java:62)
>>         at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMe
>> thodAccessorImpl.java:43)
>>         at java.lang.reflect.Method.invoke(Method.java:498)
>>         at com.linkedin.restli.internal.server.RestLiMethodInvoker.doIn
>> voke(RestLiMethodInvoker.java:186)
>>         at com.linkedin.restli.internal.server.RestLiMethodInvoker.invo
>> ke(RestLiMethodInvoker.java:141)
>>         at com.linkedin.restli.server.RestLiServer.handleResourceReques
>> t(RestLiServer.java:286)
>>         at com.linkedin.restli.server.RestLiServer.doHandleRequest(Rest
>> LiServer.java:167)
>>         at com.linkedin.restli.server.BaseRestServer.handleRequest(Base
>> RestServer.java:56)
>>         at com.linkedin.restli.server.DelegatingTransportDispatcher.han
>> dleRestRequest(DelegatingTransportDispatcher.java:56)
>>         at com.linkedin.r2.filter.transport.DispatcherRequestFilter.onR
>> estRequest(DispatcherRequestFilter.java:81)
>>         at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdap
>> ter.onRequest(FilterChainImpl.java:328)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:55)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.compression.ServerCompressionFilter.o
>> nRestRequest(ServerCompressionFilter.java:126)
>>         at com.linkedin.r2.filter.FilterChainImpl$RestRequestFilterAdap
>> ter.onRequest(FilterChainImpl.java:328)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:55)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.ComposedFilter.onRequest(ComposedFilt
>> er.java:59)
>>         at com.linkedin.r2.filter.FilterChainIterator.onRequest(FilterC
>> hainIterator.java:50)
>>         at com.linkedin.r2.filter.FilterChainImpl.onRestRequest(FilterC
>> hainImpl.java:103)
>>         at com.linkedin.r2.filter.transport.FilterChainDispatcher.handl
>> eRestRequest(FilterChainDispatcher.java:74)
>>         at com.linkedin.r2.transport.http.server.HttpDispatcher.handleR
>> equest(HttpDispatcher.java:95)
>>         at com.linkedin.r2.transport.http.server.HttpDispatcher.handleR
>> equest(HttpDispatcher.java:62)
>>         at com.linkedin.r2.transport.http.server.HttpNettyServer$Handle
>> r.messageReceived(HttpNettyServer.java:171)
>>         at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleU
>> pstream(SimpleChannelUpstreamHandler.java:80)
>>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(
>> DefaultChannelPipeline.java:545)
>>         at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChanne
>> lHandlerContext.sendUpstream(DefaultChannelPipeline.java:754)
>>         at org.jboss.netty.handler.execution.ChannelEventRunnable.run(C
>> hannelEventRunnable.java:69)
>>         at org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPo
>> olExecutor$ChildExecutor.run(OrderedMemoryAwareThreadPoolExe
>> cutor.java:316)
>>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool
>> Executor.java:1142)
>>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.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/Twitt
>> erPageExtractorPull_135/_tmp_/current.jst to
>> /home/Installable/gobblin-dist/working-dir/state-store/Twitt
>> erPageExtractorPull_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(FsDa
>> tasetStateStore.java:221)
>> at gobblin.runtime.SafeDatasetCommit.persistDatasetState(SafeDa
>> tasetCommit.java: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(ThreadPool
>> Executor.java:1142)
>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoo
>> lExecutor.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_ENABLED)))
>> {
>>       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/Configur
>> ation-Properties-Glossary/#retry-properties
>>
>> I was expecting the disabling happening by setting the follwing only
>> workunit.retry.enabled=false
>> we have to set this also
>> task.maxretries=0
>> 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=false
>>
>> 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
>>
>

Mime
View raw message