pig-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Adam Szita (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (PIG-5283) Configuration is not passed to SparkPigSplits on the backend
Date Mon, 07 Aug 2017 10:03:01 GMT

    [ https://issues.apache.org/jira/browse/PIG-5283?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16116368#comment-16116368
] 

Adam Szita commented on PIG-5283:
---------------------------------

[~kellyzly], it doesn't matter what we set in the configuration, unfortunately *Spark will
always set an empty configuration to the deserialized PigSplit*.

[~rohini], this is a problem by itself, but we didn't see it before, with Hadoop 2. Hadoop
3 on the other hand has another bug - as [~nkollar] explains - which depends on {{io.serializations}}
property being null or not. (In H2 this is handled by Hadoop code properly, in H3 it became
buggy) This however doesn't change the fact that the configuration is not passed down properly
to the deseralized PigSplits.

I don't see a way to go around the Spark code that creates the empty config during deseralization
so I intend to package the conf along with the SparkPigSplit instances.
My only question is that if we should only write those properties that are required for a
PigSplit instead of writing the full jobConf (6-700 entries) for optimization.

At first glance I see that only the following properties are needed:
{code}
CommonConfigurationKeys.IO_SERIALIZATIONS_KEY ("io.serializations")
PigConfiguration.PIG_COMPRESS_INPUT_SPLITS
{code}



> Configuration is not passed to SparkPigSplits on the backend
> ------------------------------------------------------------
>
>                 Key: PIG-5283
>                 URL: https://issues.apache.org/jira/browse/PIG-5283
>             Project: Pig
>          Issue Type: Bug
>          Components: spark
>            Reporter: Adam Szita
>            Assignee: Adam Szita
>         Attachments: PIG-5283.0.patch
>
>
> When a Hadoop ObjectWritable is created during a Spark job, the instantiated PigSplit
(wrapped into a SparkPigSplit) is given an empty Configuration instance.
> This happens [here|https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SerializableWritable.scala#L44]



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message