Hi, the Jira Bug is here: https://issues.apache.org/jira/browse/SPARK-23799 I have also created the PR for the issue: https://github.com/apache/spark/pull/20913 With this fix, it is working for me really well.
Best, Michael On Sat, Mar 24, 2018 at 12:39 AM, Takeshi Yamamuro <linguin....@gmail.com> wrote: > Can you file a jira if this is a bug? > Thanks! > > On Sat, Mar 24, 2018 at 1:23 AM, Michael Shtelma <mshte...@gmail.com> wrote: >> >> Hi Maropu, >> >> the problem seems to be in FilterEstimation.scala on lines 50 and 52: >> >> https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala?utf8=✓#L50-L52 >> >> val filterSelectivity = >> calculateFilterSelectivity(plan.condition).getOrElse(1.0) >> val filteredRowCount: BigInt = >> ceil(BigDecimal(childStats.rowCount.get) * filterSelectivity) >> >> The problem is, that filterSelectivity gets NaN value in my case and >> NaN cannot be converted to BigDecimal. >> I can try adding simple if, checking the NaN value and test if this helps. >> I will also try to understand, why in my case, I am getting NaN. >> >> Best, >> Michael >> >> >> On Fri, Mar 23, 2018 at 1:51 PM, Takeshi Yamamuro <linguin....@gmail.com> >> wrote: >> > hi, >> > >> > What's a query to reproduce this? >> > It seems when casting double to BigDecimal, it throws the exception. >> > >> > // maropu >> > >> > On Fri, Mar 23, 2018 at 6:20 PM, Michael Shtelma <mshte...@gmail.com> >> > wrote: >> >> >> >> Hi all, >> >> >> >> I am using Spark 2.3 with activated cost-based optimizer and a couple >> >> of hive tables, that were analyzed previously. >> >> >> >> I am getting the following exception for different queries: >> >> >> >> java.lang.NumberFormatException >> >> >> >> at java.math.BigDecimal.<init>(BigDecimal.java:494) >> >> >> >> at java.math.BigDecimal.<init>(BigDecimal.java:824) >> >> >> >> at scala.math.BigDecimal$.decimal(BigDecimal.scala:52) >> >> >> >> at scala.math.BigDecimal$.decimal(BigDecimal.scala:55) >> >> >> >> at scala.math.BigDecimal$.double2bigDecimal(BigDecimal.scala:343) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.FilterEstimation.estimate(FilterEstimation.scala:52) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visitFilter(BasicStatsPlanVisitor.scala:43) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visitFilter(BasicStatsPlanVisitor.scala:25) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.LogicalPlanVisitor$class.visit(LogicalPlanVisitor.scala:30) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visit(BasicStatsPlanVisitor.scala:25) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$$anonfun$stats$1.apply(LogicalPlanStats.scala:35) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$$anonfun$stats$1.apply(LogicalPlanStats.scala:33) >> >> >> >> at scala.Option.getOrElse(Option.scala:121) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$class.stats(LogicalPlanStats.scala:33) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.stats(LogicalPlan.scala:30) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils$$anonfun$rowCountsExist$1.apply(EstimationUtils.scala:32) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils$$anonfun$rowCountsExist$1.apply(EstimationUtils.scala:32) >> >> >> >> at >> >> >> >> scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38) >> >> >> >> at >> >> >> >> scala.collection.IndexedSeqOptimized$class.forall(IndexedSeqOptimized.scala:43) >> >> >> >> at scala.collection.mutable.WrappedArray.forall(WrappedArray.scala:35) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils$.rowCountsExist(EstimationUtils.scala:32) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.ProjectEstimation$.estimate(ProjectEstimation.scala:27) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visitProject(BasicStatsPlanVisitor.scala:63) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visitProject(BasicStatsPlanVisitor.scala:25) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.LogicalPlanVisitor$class.visit(LogicalPlanVisitor.scala:37) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.BasicStatsPlanVisitor$.visit(BasicStatsPlanVisitor.scala:25) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$$anonfun$stats$1.apply(LogicalPlanStats.scala:35) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$$anonfun$stats$1.apply(LogicalPlanStats.scala:33) >> >> >> >> at scala.Option.getOrElse(Option.scala:121) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.statsEstimation.LogicalPlanStats$class.stats(LogicalPlanStats.scala:33) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.stats(LogicalPlan.scala:30) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$$anonfun$2.apply(CostBasedJoinReorder.scala:64) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$$anonfun$2.apply(CostBasedJoinReorder.scala:64) >> >> >> >> at >> >> >> >> scala.collection.LinearSeqOptimized$class.forall(LinearSeqOptimized.scala:83) >> >> >> >> at scala.collection.immutable.List.forall(List.scala:84) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$.org$apache$spark$sql$catalyst$optimizer$CostBasedJoinReorder$$reorder(CostBasedJoinReorder.scala:64) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$$anonfun$1.applyOrElse(CostBasedJoinReorder.scala:46) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$$anonfun$1.applyOrElse(CostBasedJoinReorder.scala:43) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4$$anonfun$apply$11.apply(TreeNode.scala:335) >> >> >> >> at >> >> >> >> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) >> >> >> >> at >> >> >> >> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) >> >> >> >> at scala.collection.immutable.List.foreach(List.scala:392) >> >> >> >> at >> >> scala.collection.TraversableLike$class.map(TraversableLike.scala:234) >> >> >> >> at scala.collection.immutable.List.map(List.scala:296) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:333) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4$$anonfun$apply$11.apply(TreeNode.scala:335) >> >> >> >> at >> >> >> >> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) >> >> >> >> at >> >> >> >> scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) >> >> >> >> at scala.collection.immutable.List.foreach(List.scala:392) >> >> >> >> at >> >> scala.collection.TraversableLike$class.map(TraversableLike.scala:234) >> >> >> >> at scala.collection.immutable.List.map(List.scala:296) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:333) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$.apply(CostBasedJoinReorder.scala:43) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.optimizer.CostBasedJoinReorder$.apply(CostBasedJoinReorder.scala:35) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:87) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:84) >> >> >> >> at >> >> >> >> scala.collection.IndexedSeqOptimized$class.foldl(IndexedSeqOptimized.scala:57) >> >> >> >> at >> >> >> >> scala.collection.IndexedSeqOptimized$class.foldLeft(IndexedSeqOptimized.scala:66) >> >> >> >> at >> >> scala.collection.mutable.WrappedArray.foldLeft(WrappedArray.scala:35) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:84) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:76) >> >> >> >> at scala.collection.immutable.List.foreach(List.scala:392) >> >> >> >> at >> >> >> >> org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:76) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:66) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:66) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:204) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution$$anonfun$toString$2.apply(QueryExecution.scala:204) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution.stringOrError(QueryExecution.scala:100) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:204) >> >> >> >> at >> >> >> >> org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:74) >> >> >> >> at >> >> >> >> org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:654) >> >> >> >> at >> >> >> >> org.apache.spark.sql.DataFrameWriter.createTable(DataFrameWriter.scala:458) >> >> >> >> at >> >> >> >> org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:437) >> >> >> >> at >> >> >> >> org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:393) >> >> >> >> >> >> >> >> This exception only comes, if the statistics exist for the hive tables >> >> being used. >> >> >> >> Has anybody already seen something like this ? >> >> Any assistance would be greatly appreciated! >> >> >> >> Best, >> >> Michael >> >> >> >> --------------------------------------------------------------------- >> >> To unsubscribe e-mail: user-unsubscr...@spark.apache.org >> >> >> > >> > >> > >> > -- >> > --- >> > Takeshi Yamamuro > > > > > -- > --- > Takeshi Yamamuro --------------------------------------------------------------------- To unsubscribe e-mail: user-unsubscr...@spark.apache.org