You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@kylin.apache.org by "Richard Calaba (JIRA)" <ji...@apache.org> on 2016/05/04 00:53:12 UTC

[jira] [Created] (KYLIN-1650) java.io.IOException: Filesystem closed - in Cube Build Step 2

Richard Calaba created KYLIN-1650:
-------------------------------------

             Summary: java.io.IOException: Filesystem closed - in Cube Build Step 2 
                 Key: KYLIN-1650
                 URL: https://issues.apache.org/jira/browse/KYLIN-1650
             Project: Kylin
          Issue Type: Bug
    Affects Versions: v1.5.1, v1.5.0
         Environment: MapR 4.1
            Reporter: Richard Calaba


Hello,

I am facing error in Build Cube - Step 2 - 4 Reducers are ending with excpetion java.io.IOException: Filesystem closed .... I have searched a lot and tried various versions (1.5.1, 1.5.0 - binary / compiled from sources ...) and always with same result ... all on MapR 4.1 Hadoop

The problem seems related also to data ... because exactly same cube defined in different Hive schema (DB) - with exactly same metadata - worked fine. BUT trying to the the fact table and lookup table join (one lookup table only) on the Hive tables as defined in the failing Cube Model works fine !!!! So if it is data issue -> then it is strange that select count(*) from fact left outer lookup works without issues ... 

Attaching syslog of one failed reducers ... in case anyone gets an idea what could be the cause of this weird error ...


Log Type: stderr
Log Length: 507
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in [jar:file:/data/mapr/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/filecache/10/job.jar/job.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]

Log Type: stdout
Log Length: 304
spill values to disk...
create file /kylin/kylin_metadata/kylin-7c7f995d-4925-4fcc-b0bc-c4a97a547c2c/cube_xxxxx_model_150_2ndattempt/fact_distinct_columns/BILLDATE
append file /kylin/kylin_metadata/kylin-7c7f995d-4925-4fcc-b0bc-c4a97a547c2c/cube_xxxxx_model_150_2ndattempt/fact_distinct_columns/BILLDATE

Log Type: syslog
Log Length: 10167
Showing 4096 bytes of 10167 total. Click here for the full log.
es from disk
2016-05-03 18:38:11,275 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: Merging 0 segments, 0 bytes from memory into reduce
2016-05-03 18:38:11,275 INFO [main] org.apache.hadoop.mapred.Merger: Merging 1 sorted segments
2016-05-03 18:38:11,279 INFO [main] org.apache.hadoop.mapred.Merger: Down to the last merge-pass, with 1 segments left of total size: 8605536 bytes
2016-05-03 18:38:11,322 INFO [main] org.apache.hadoop.conf.Configuration.deprecation: mapred.skip.on is deprecated. Instead, use mapreduce.job.skiprecords
2016-05-03 18:38:11,342 INFO [main] org.apache.kylin.engine.mr.common.AbstractHadoopJob: The absolute path for meta dir is /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,347 INFO [main] org.apache.kylin.common.KylinConfig: Use KYLIN_CONF=/tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,352 INFO [main] org.apache.kylin.common.KylinConfigBase: Kylin Config was updated with kylin.metadata.url : /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,371 INFO [main] org.apache.kylin.cube.CubeManager: Initializing CubeManager with config /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,376 INFO [main] org.apache.kylin.common.persistence.ResourceStore: Using metadata url /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta for resource store
2016-05-03 18:38:11,380 WARN [main] org.apache.kylin.common.persistence.ResourceStore: Failed to load ResourceStore impl class: java.lang.NoClassDefFoundError: org/apache/hadoop/hbase/filter/Filter
2016-05-03 18:38:11,772 INFO [main] org.apache.kylin.cube.CubeDescManager: Initializing CubeDescManager with config /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,772 INFO [main] org.apache.kylin.cube.CubeDescManager: Reloading Cube Metadata from folder /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta/cube_desc
2016-05-03 18:38:14,530 WARN [main] org.apache.hadoop.mapred.YarnChild: Exception running child : java.io.IOException: Filesystem closed
	at com.mapr.fs.MapRFileSystem.checkOpen(MapRFileSystem.java:1469)
	at com.mapr.fs.MapRFileSystem.lookupClient(MapRFileSystem.java:555)
	at com.mapr.fs.MapRFileSystem.lookupClient(MapRFileSystem.java:649)
	at com.mapr.fs.MapRFileSystem.delete(MapRFileSystem.java:1048)
	at org.apache.hadoop.mapred.Merger$Segment.close(Merger.java:348)
	at org.apache.hadoop.mapred.Merger$MergeQueue.adjustPriorityQueue(Merger.java:521)
	at org.apache.hadoop.mapred.Merger$MergeQueue.next(Merger.java:533)
	at org.apache.hadoop.mapred.ReduceTask$4.next(ReduceTask.java:601)
	at org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKeyValue(ReduceContextImpl.java:155)
	at org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKey(ReduceContextImpl.java:121)
	at org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer$Context.nextKey(WrappedReducer.java:302)
	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:170)
	at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:627)
	at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:389)
	at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:415)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1566)
	at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163)

2016-05-03 18:38:14,536 INFO [main] org.apache.hadoop.mapred.Task: Runnning cleanup for the task



ALSO ADDING FULL SYSLOG:
=========================

Log Type: syslog
Log Length: 10167
2016-05-03 18:38:05,984 WARN [main] org.apache.hadoop.conf.Configuration: job.xml:an attempt to override final parameter: mapreduce.job.end-notification.max.retry.interval;  Ignoring.
2016-05-03 18:38:05,986 WARN [main] org.apache.hadoop.conf.Configuration: job.xml:an attempt to override final parameter: mapreduce.job.end-notification.max.attempts;  Ignoring.
2016-05-03 18:38:06,213 INFO [main] org.apache.hadoop.metrics2.impl.MetricsConfig: loaded properties from hadoop-metrics2.properties
2016-05-03 18:38:06,314 INFO [main] org.apache.hadoop.metrics2.impl.MetricsSystemImpl: Scheduled snapshot period at 10 second(s).
2016-05-03 18:38:06,314 INFO [main] org.apache.hadoop.metrics2.impl.MetricsSystemImpl: ReduceTask metrics system started
2016-05-03 18:38:06,435 INFO [main] org.apache.hadoop.mapred.YarnChild: Executing with tokens:
2016-05-03 18:38:06,484 INFO [main] org.apache.hadoop.mapred.YarnChild: Kind: mapreduce.job, Service: job_1461856462442_0134, Ident: (org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier@6bb31fae)
2016-05-03 18:38:06,536 INFO [main] org.apache.hadoop.mapred.YarnChild: Sleeping for 0ms before retrying again. Got null now.
2016-05-03 18:38:06,842 INFO [main] org.apache.hadoop.mapred.YarnChild: mapreduce.cluster.local.dir for child: /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134
2016-05-03 18:38:06,999 WARN [main] org.apache.hadoop.conf.Configuration: job.xml:an attempt to override final parameter: mapreduce.job.end-notification.max.retry.interval;  Ignoring.
2016-05-03 18:38:07,000 WARN [main] org.apache.hadoop.conf.Configuration: job.xml:an attempt to override final parameter: mapreduce.job.end-notification.max.attempts;  Ignoring.
2016-05-03 18:38:07,481 INFO [main] org.apache.hadoop.mapred.Task: mapOutputFile class: org.apache.hadoop.mapred.MapRFsOutputFile
2016-05-03 18:38:07,481 INFO [main] org.apache.hadoop.conf.Configuration.deprecation: session.id is deprecated. Instead, use dfs.metrics.session-id
2016-05-03 18:38:07,521 INFO [main] org.apache.hadoop.mapred.Task:  Using ResourceCalculatorProcessTree : [ ]
2016-05-03 18:38:07,603 INFO [main] org.apache.hadoop.mapred.ReduceTask: Using ShuffleConsumerPlugin: org.apache.hadoop.mapreduce.task.reduce.DirectShuffle@728cacdc
2016-05-03 18:38:07,632 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: MergerManager: memoryLimit=1503238528, maxSingleShuffleLimit=375809632, mergeThreshold=992137472, ioSortFactor=256, memToMemMergeOutputsThreshold=256
2016-05-03 18:38:07,638 INFO [EventFetcher for fetching Map Completion Events] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleEventFetcher: attempt_1461856462442_0134_r_000000_0 Thread started: EventFetcher for fetching Map Completion Events
2016-05-03 18:38:07,656 INFO [EventFetcher for fetching Map Completion Events] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleEventFetcher: attempt_1461856462442_0134_r_000000_0: Got 1 new map-outputs
2016-05-03 18:38:07,685 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,692 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,693 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,695 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,696 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,697 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,698 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,699 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,700 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,702 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,703 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,705 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,767 INFO [MapOutputCopier task_1461856462442_0134_r_000000.0] org.apache.hadoop.io.compress.CodecPool: Got brand-new decompressor [.snappy]
2016-05-03 18:38:07,770 INFO [MapOutputCopier task_1461856462442_0134_r_000000.0] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleFetcher: fetcher#0 about to shuffle output of map attempt_1461856462442_0134_m_000000_0 to MEMORY
2016-05-03 18:38:08,046 INFO [MapOutputCopier task_1461856462442_0134_r_000000.0] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: closeInMemoryFile -> map-output of size: 52779014, inMemoryMapOutputs.size() -> 1, commitMemory -> 0, usedMemory ->52779014
2016-05-03 18:38:08,047 INFO [EventFetcher for fetching Map Completion Events] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleEventFetcher: EventFetcher is interrupted.. Returning
2016-05-03 18:38:08,141 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: finalMerge called with 1 in-memory map-outputs and 0 on-disk map-outputs
2016-05-03 18:38:08,151 INFO [main] org.apache.hadoop.mapred.Merger: Merging 1 sorted segments
2016-05-03 18:38:08,152 INFO [main] org.apache.hadoop.mapred.Merger: Down to the last merge-pass, with 1 segments left of total size: 52778982 bytes
2016-05-03 18:38:08,160 INFO [main] org.apache.hadoop.io.compress.CodecPool: Got brand-new compressor [.snappy]
2016-05-03 18:38:11,273 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: Merged 1 segments, 52779014 bytes to disk to satisfy reduce memory limit
2016-05-03 18:38:11,274 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: Merging 1 files, 8644203 bytes from disk
2016-05-03 18:38:11,275 INFO [main] org.apache.hadoop.mapreduce.task.reduce.DirectShuffleMergeManagerImpl: Merging 0 segments, 0 bytes from memory into reduce
2016-05-03 18:38:11,275 INFO [main] org.apache.hadoop.mapred.Merger: Merging 1 sorted segments
2016-05-03 18:38:11,279 INFO [main] org.apache.hadoop.mapred.Merger: Down to the last merge-pass, with 1 segments left of total size: 8605536 bytes
2016-05-03 18:38:11,322 INFO [main] org.apache.hadoop.conf.Configuration.deprecation: mapred.skip.on is deprecated. Instead, use mapreduce.job.skiprecords
2016-05-03 18:38:11,342 INFO [main] org.apache.kylin.engine.mr.common.AbstractHadoopJob: The absolute path for meta dir is /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,347 INFO [main] org.apache.kylin.common.KylinConfig: Use KYLIN_CONF=/tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,352 INFO [main] org.apache.kylin.common.KylinConfigBase: Kylin Config was updated with kylin.metadata.url : /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,371 INFO [main] org.apache.kylin.cube.CubeManager: Initializing CubeManager with config /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,376 INFO [main] org.apache.kylin.common.persistence.ResourceStore: Using metadata url /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta for resource store
2016-05-03 18:38:11,380 WARN [main] org.apache.kylin.common.persistence.ResourceStore: Failed to load ResourceStore impl class: java.lang.NoClassDefFoundError: org/apache/hadoop/hbase/filter/Filter
2016-05-03 18:38:11,772 INFO [main] org.apache.kylin.cube.CubeDescManager: Initializing CubeDescManager with config /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta
2016-05-03 18:38:11,772 INFO [main] org.apache.kylin.cube.CubeDescManager: Reloading Cube Metadata from folder /tmp/hadoop-mapr/nm-local-dir/usercache/mapr/appcache/application_1461856462442_0134/container_1461856462442_0134_01_000003/meta/cube_desc
2016-05-03 18:38:14,530 WARN [main] org.apache.hadoop.mapred.YarnChild: Exception running child : java.io.IOException: Filesystem closed
	at com.mapr.fs.MapRFileSystem.checkOpen(MapRFileSystem.java:1469)
	at com.mapr.fs.MapRFileSystem.lookupClient(MapRFileSystem.java:555)
	at com.mapr.fs.MapRFileSystem.lookupClient(MapRFileSystem.java:649)
	at com.mapr.fs.MapRFileSystem.delete(MapRFileSystem.java:1048)
	at org.apache.hadoop.mapred.Merger$Segment.close(Merger.java:348)
	at org.apache.hadoop.mapred.Merger$MergeQueue.adjustPriorityQueue(Merger.java:521)
	at org.apache.hadoop.mapred.Merger$MergeQueue.next(Merger.java:533)
	at org.apache.hadoop.mapred.ReduceTask$4.next(ReduceTask.java:601)
	at org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKeyValue(ReduceContextImpl.java:155)
	at org.apache.hadoop.mapreduce.task.ReduceContextImpl.nextKey(ReduceContextImpl.java:121)
	at org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer$Context.nextKey(WrappedReducer.java:302)
	at org.apache.hadoop.mapreduce.Reducer.run(Reducer.java:170)
	at org.apache.hadoop.mapred.ReduceTask.runNewReducer(ReduceTask.java:627)
	at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:389)
	at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:168)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:415)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1566)
	at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:163)

2016-05-03 18:38:14,536 INFO [main] org.apache.hadoop.mapred.Task: Runnning cleanup for the task




--
This message was sent by Atlassian JIRA
(v6.3.4#6332)