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

ASF GitHub Bot commented on CARBONDATA-300:
-------------------------------------------

Github user jackylk commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/244#discussion_r84443403
  
    --- Diff: 
processing/src/main/java/org/apache/carbondata/processing/newflow/encoding/impl/RowConverterImpl.java
 ---
    @@ -0,0 +1,94 @@
    +/*
    + * 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.processing.newflow.encoding.impl;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.carbondata.core.cache.Cache;
    +import org.apache.carbondata.core.cache.CacheProvider;
    +import org.apache.carbondata.core.cache.CacheType;
    +import org.apache.carbondata.core.cache.dictionary.Dictionary;
    +import 
org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
    +import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
    +import 
org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
    +import org.apache.carbondata.processing.newflow.DataField;
    +import 
org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
    +import org.apache.carbondata.processing.newflow.encoding.FieldConverter;
    +import org.apache.carbondata.processing.newflow.encoding.RowConverter;
    +import 
org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
    +import org.apache.carbondata.processing.newflow.row.CarbonRow;
    +
    +import org.apache.commons.lang3.ArrayUtils;
    +
    +/**
    + *
    + */
    +public class RowConverterImpl implements RowConverter {
    +
    +  private CarbonDataLoadConfiguration configuration;
    +
    +  private FieldConverter[] fieldConverters;
    +
    +  public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration 
configuration) {
    +    this.configuration = configuration;
    +    CacheProvider cacheProvider = CacheProvider.getInstance();
    +    Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache =
    +        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY,
    +            configuration.getTableIdentifier().getStorePath());
    +    List<FieldConverter> fieldConverterList = new ArrayList<>();
    +
    +    long lruCacheStartTime = System.currentTimeMillis();
    +
    +    for (int i = 0; i < fields.length; i++) {
    +      FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
    +          .createFieldEncoder(fields[i], cache,
    +              
configuration.getTableIdentifier().getCarbonTableIdentifier(), i);
    +      if (fieldConverter != null) {
    +        fieldConverterList.add(fieldConverter);
    +      }
    +    }
    +    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
    +        .recordLruCacheLoadTime((System.currentTimeMillis() - 
lruCacheStartTime) / 1000.0);
    +    fieldConverters = fieldConverterList.toArray(new 
FieldConverter[fieldConverterList.size()]);
    +  }
    +
    +  @Override
    +  public CarbonRow convert(CarbonRow row) throws 
CarbonDataLoadingException {
    +
    +    for (int i = 0; i < fieldConverters.length; i++) {
    +      fieldConverters[i].convert(row);
    +    }
    +    return row;
    +  }
    +
    +  @Override
    +  public void finish() {
    +    List<Integer> dimCardinality = new ArrayList<>();
    +    for (int i = 0; i < fieldConverters.length; i++) {
    +      if (fieldConverters[i] instanceof 
AbstractDictionaryFieldConverterImpl) dimCardinality
    +          .add(((AbstractDictionaryFieldConverterImpl) 
fieldConverters[i]).getColumnCardinality());
    +    }
    +    int[] cardinality =
    +        ArrayUtils.toPrimitive(dimCardinality.toArray(new 
Integer[dimCardinality.size()]));
    --- End diff --
    
    this will copy twice, better to loop and convert itself


> 5. Add EncodeProcessorStep which encodes the data with dictionary.
> ------------------------------------------------------------------
>
>                 Key: CARBONDATA-300
>                 URL: https://issues.apache.org/jira/browse/CARBONDATA-300
>             Project: CarbonData
>          Issue Type: Sub-task
>            Reporter: Ravindra Pesala
>            Assignee: Ravindra Pesala
>             Fix For: 0.2.0-incubating
>
>
> Add EncodeProcessorStep which encodes the data with dictionary.This 
> dictionary can be obtained from dictionary interface.



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

Reply via email to