carbondata-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jackylk <...@git.apache.org>
Subject [GitHub] carbondata pull request #910: [WIP] Global sort by spark in load process
Date Mon, 12 Jun 2017 02:47:36 GMT
Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/910#discussion_r121302968
  
    --- Diff: integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortOperates.scala
---
    @@ -0,0 +1,242 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.carbondata.spark.load
    +
    +import org.apache.carbondata.common.logging.LogServiceFactory
    +import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
    +import org.apache.carbondata.core.datastore.row.CarbonRow
    +import org.apache.carbondata.core.util.CarbonProperties
    +import org.apache.carbondata.processing.csvload.StringArrayWritable
    +import org.apache.carbondata.processing.model.CarbonLoadModel
    +import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
    +import org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
    +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
    +import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
    +import org.apache.carbondata.processing.newflow.sort.SortHelper
    +import org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl,
DataWriterProcessorStepImpl}
    +import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
    +import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
    +import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
    +import org.apache.spark.broadcast.Broadcast
    +import org.apache.spark.sql.Row
    +import org.apache.spark.util.LongAccumulator
    +import org.apache.spark.{SparkEnv, TaskContext}
    +
    +import scala.util.Random
    +
    +object GlobalSortOperates {
    +  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
    +
    +  def toStringArrayRow(row: StringArrayWritable, columnCount: Int): StringArrayRow =
{
    +    val outRow = new StringArrayRow(new Array[String](columnCount))
    +    outRow.setValues(row.get())
    +  }
    +
    +  def toRDDIterator(
    +      rows: Iterator[Row],
    +      modelBroadcast: Broadcast[CarbonLoadModel]): Iterator[Array[AnyRef]] = {
    +    new Iterator[Array[AnyRef]] {
    +      val iter = new NewRddIterator(rows, modelBroadcast.value, TaskContext.get())
    +
    +      override def hasNext: Boolean = iter.hasNext
    +
    +      override def next(): Array[AnyRef] = iter.next
    +    }
    +  }
    +
    +  def inputFunc(
    +      rows: Iterator[Array[AnyRef]],
    +      index: Int,
    +      currentLoadCount: Int,
    +      modelBroadcast: Broadcast[CarbonLoadModel],
    +      rowNumber: LongAccumulator): Iterator[CarbonRow] = {
    +    val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, modelBroadcast)
    +    val conf = DataLoadProcessBuilder.createConfiguration(model)
    +    val rowParser = new RowParserImpl(conf.getDataFields, conf)
    +
    +    TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
    +      wrapException(e, model)
    +    }
    +
    +    new Iterator[CarbonRow] {
    +      override def hasNext: Boolean = rows.hasNext
    +
    +      override def next(): CarbonRow = {
    +        rowNumber.add(1)
    +        new CarbonRow(rowParser.parseRow(rows.next()))
    +      }
    +    }
    +  }
    +
    +  def convertFunc(
    +      rows: Iterator[CarbonRow],
    +      index: Int,
    +      currentLoadCount: Int,
    +      modelBroadcast: Broadcast[CarbonLoadModel],
    +      partialSuccessAccum: LongAccumulator,
    +      rowNumber: LongAccumulator): Iterator[CarbonRow] = {
    +    val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, modelBroadcast)
    +    val conf = DataLoadProcessBuilder.createConfiguration(model)
    +    val badRecordLogger = DataConverterProcessorStepImpl.createBadRecordLogger(conf)
    +    val rowConverter = new RowConverterImpl(conf.getDataFields, conf, badRecordLogger)
    +    rowConverter.initialize()
    +
    +    TaskContext.get().addTaskCompletionListener { context =>
    +      DataConverterProcessorStepImpl.close(badRecordLogger, conf, rowConverter)
    +      GlobalSortHelper.badRecordsLogger(model, partialSuccessAccum)
    +    }
    +
    +    TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
    +      DataConverterProcessorStepImpl.close(badRecordLogger, conf, rowConverter)
    +      GlobalSortHelper.badRecordsLogger(model, partialSuccessAccum)
    +
    +      wrapException(e, model)
    +    }
    +
    +    new Iterator[CarbonRow] {
    +      override def hasNext: Boolean = rows.hasNext
    +
    +      override def next(): CarbonRow = {
    +        rowNumber.add(1)
    +        rowConverter.convert(rows.next())
    +      }
    +    }
    +  }
    +
    +  def convertTo3Parts(
    +      rows: Iterator[CarbonRow],
    +      index: Int,
    +      currentLoadCount: Int,
    +      modelBroadcast: Broadcast[CarbonLoadModel],
    +      rowNumber: LongAccumulator): Iterator[CarbonRow] = {
    +    val model: CarbonLoadModel = getModelCopy(index, currentLoadCount, modelBroadcast)
    +    val conf = DataLoadProcessBuilder.createConfiguration(model)
    +    val sortParameters = SortParameters.createSortParameters(conf)
    +    val sortHelper = new SortHelper(sortParameters)
    +
    +    TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
    +      wrapException(e, model)
    +    }
    +
    +    new Iterator[CarbonRow] {
    +      override def hasNext: Boolean = rows.hasNext
    +
    +      override def next(): CarbonRow = {
    +        rowNumber.add(1)
    +        new CarbonRow(sortHelper.convertRow(rows.next().getData))
    +      }
    +    }
    +  }
    +
    +  def writeFunc(
    +      rows: Iterator[CarbonRow],
    +      index: Int,
    +      currentLoadCount: Int,
    +      modelBroadcast: Broadcast[CarbonLoadModel],
    +      rowNumber: LongAccumulator) {
    +    var model: CarbonLoadModel = null
    +    var tableName: String = null
    +    var rowConverter: RowConverterImpl = null
    +
    +    try {
    +      model = getModelCopy(index, currentLoadCount, modelBroadcast)
    +      val storeLocation = getTempStoreLocation(index)
    +      val conf = DataLoadProcessBuilder.createConfiguration(model, storeLocation)
    +
    +      tableName = model.getTableName
    +
    +      // When we use sortBy, it means we have 2 stages. Stage1 can't get the finder from
Stage2 directly because they
    +      // are in different processes. We need to set cardinality finder in Stage1 again.
    +      rowConverter = new RowConverterImpl(conf.getDataFields, conf, null)
    +      rowConverter.initialize()
    +      conf.setCardinalityFinder(rowConverter)
    +
    +      val dataWriter = new DataWriterProcessorStepImpl(conf)
    +
    +      val dataHandlerModel = dataWriter.getDataHandlerModel(0)
    +      var dataHandler: CarbonFactHandler = null
    +      var rowsNotExist = true
    +      while (rows.hasNext) {
    +        if (rowsNotExist) {
    +          rowsNotExist = false
    +          dataHandler = CarbonFactHandlerFactory.createCarbonFactHandler(dataHandlerModel,
    +            CarbonFactHandlerFactory.FactHandlerType.COLUMNAR)
    +          dataHandler.initialise()
    +        }
    +        rowNumber.add(1)
    +        dataWriter.processRow(rows.next(), dataHandler)
    +      }
    +
    +      if (!rowsNotExist) {
    +        dataWriter.finish(dataHandler)
    +      }
    +    } catch {
    +      case e: CarbonDataWriterException =>
    +        LOGGER.error(e, "Failed for table: " + tableName + " in Data Writer Step")
    +        throw new CarbonDataLoadingException("Error while initializing data handler :
" + e.getMessage)
    +      case e: Exception =>
    +        LOGGER.error(e, "Failed for table: " + tableName + " in Data Writer Step")
    +        throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage,
e)
    +    } finally {
    +      if (rowConverter != null) {
    +        rowConverter.finish()
    +      }
    +      // clean up the folders and files created locally for data load operation
    +      CarbonLoaderUtil.deleteLocalDataLoadFolderLocation(model, false)
    --- End diff --
    
    suggest to move this clean up to caller of `writeFunc`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message