[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user asfgit closed the pull request at: https://github.com/apache/incubator-carbondata/pull/244 --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84518288 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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); --- End diff -- Yes here fields are same as input, the 3 elements code is moved to sort step. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84513301 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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 { + --- End diff -- remove empty line --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84513452 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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); --- End diff -- so, after convert, it is different from old approach which just have 3 elements, right? here, the number of fields remain the same as input. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84512153 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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()]); --- End diff -- ok --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84508220 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java --- @@ -0,0 +1,59 @@ +/* + * 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.converter.impl; + +import org.apache.carbondata.core.cache.Cache; +import org.apache.carbondata.core.cache.dictionary.Dictionary; +import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier; +import org.apache.carbondata.core.carbon.CarbonTableIdentifier; +import org.apache.carbondata.core.carbon.metadata.encoder.Encoding; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.converter.FieldConverter; + +public class FieldEncoderFactory { + + private static FieldEncoderFactory instance; + + private FieldEncoderFactory() { + + } + + public static FieldEncoderFactory getInstance() { +if (instance == null) { + instance = new FieldEncoderFactory(); +} +return instance; + } + + public FieldConverter createFieldEncoder(DataField dataField, --- End diff -- added --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84508170 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java --- @@ -0,0 +1,59 @@ +/* + * 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.converter.impl; + +import org.apache.carbondata.core.cache.Cache; +import org.apache.carbondata.core.cache.dictionary.Dictionary; +import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier; +import org.apache.carbondata.core.carbon.CarbonTableIdentifier; +import org.apache.carbondata.core.carbon.metadata.encoder.Encoding; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.converter.FieldConverter; + +public class FieldEncoderFactory { + + private static FieldEncoderFactory instance; + + private FieldEncoderFactory() { + + } + + public static FieldEncoderFactory getInstance() { +if (instance == null) { + instance = new FieldEncoderFactory(); +} +return instance; + } + + public FieldConverter createFieldEncoder(DataField dataField, + Cachecache, + CarbonTableIdentifier carbonTableIdentifier, int index) { +if (dataField.hasDictionaryEncoding()) { + return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, index); +} else if (dataField.getColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) { + return new DirectDictionaryFieldConverterImpl(dataField, index); +} else if (dataField.getColumn().isComplex()) { + return new ComplexFieldConverterImpl(); +} else if ((dataField.getColumn().hasEncoding(Encoding.DICTIONARY) || --- End diff -- ok --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84504071 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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()]); --- End diff -- We can use iterator, but it would be called each row, so accessing array is much faster than arrayList. And regarding copy is a one time operation. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84484251 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java --- @@ -0,0 +1,99 @@ +/* + * 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.converter.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.converter.FieldConverter; +import org.apache.carbondata.processing.newflow.converter.RowConverter; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +import org.apache.commons.lang3.ArrayUtils; + +/** + * It converts the complete row if necessary, dictionary columns are encoded with dictionary values + * and nondictionary values are converted to binary. + */ +public class RowConverterImpl implements RowConverter { + + private CarbonDataLoadConfiguration configuration; + + private FieldConverter[] fieldConverters; + + public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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()]); --- End diff -- but in this case it needs one round of copy. and using list you can also iterate using its iterator --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84482988 --- 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(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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); + } --- End diff -- ok. only dictionary and no dictionary and complex column has converter, right? can we make this code more readable, like checking whether it is non-measure. And measure can not be complex column? --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84474110 --- 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(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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); + } --- End diff -- No @jackylk , because for measures we don't have any converters so it returns null for that case. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84472920 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/encoding/RowEncoder.java --- @@ -0,0 +1,32 @@ +/* + * 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; + +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Encodes the row + */ +public interface RowEncoder { --- End diff -- Ok, I will rename the Encoder to converter --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user ravipesala commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84473138 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/encodestep/EncoderProcessorStepImpl.java --- @@ -0,0 +1,63 @@ +/* + * 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.steps.encodestep; + +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep; +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.encoding.RowEncoder; +import org.apache.carbondata.processing.newflow.encoding.impl.RowEncoderImpl; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Encode data with dictionary values and composed with bit/byte packed key. + * nondictionary values are packed as bytes, and complex types are also packed as bytes. + */ +public class EncoderProcessorStepImpl extends AbstractDataLoadProcessorStep { --- End diff -- Here I convert only dictionary and nondictionary fields, remaining preparation is moved to Sort step. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84443937 --- 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(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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); + } --- End diff -- if it is null, should throw exception --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84443155 --- 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(); +Cachecache = +cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List 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 dimCardinality = new ArrayList<>(); +for (int i = 0; i < fieldConverters.length; i++) { + if (fieldConverters[i] instanceof AbstractDictionaryFieldConverterImpl) dimCardinality --- End diff -- incorrect coding style --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84443496 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java --- @@ -0,0 +1,61 @@ +/* + * 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.steps; + +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep; +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.encoding.RowConverter; +import org.apache.carbondata.processing.newflow.encoding.impl.RowConverterImpl; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Replace row data fields with dictionary values if column is configured dictionary encoded. + * And nondictionary columns as well as complex columns will be converted to byte[]. + */ +public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorStep { + + private RowConverter encoder; + + public DataConverterProcessorStepImpl(CarbonDataLoadConfiguration configuration, + AbstractDataLoadProcessorStep child) { +super(configuration, child); + } + + @Override public DataField[] getOutput() { --- End diff -- move override to next line --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84443007 --- 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; + +/** + * --- End diff -- please add description --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84243237 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/encoding/impl/NonDictionaryFieldEncoderImpl.java --- @@ -0,0 +1,54 @@ +/* + * 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.nio.ByteBuffer; +import java.nio.charset.Charset; + +import org.apache.carbondata.core.carbon.metadata.datatype.DataType; +import org.apache.carbondata.core.constants.CarbonCommonConstants; +import org.apache.carbondata.core.util.DataTypeUtil; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.encoding.FieldEncoder; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +public class NonDictionaryFieldEncoderImpl implements FieldEncoder { + + private DataType dataType; + + private int index; + + public NonDictionaryFieldEncoderImpl(DataField dataField, int index) { +this.dataType = dataField.getColumn().getDataType(); +this.index = index; + } + + @Override public ByteBuffer encode(CarbonRow row) { +String dimensionValue = row.getString(index); +if (dataType != DataType.STRING) { + if (null == DataTypeUtil.normalizeIntAndLongValues(dimensionValue, dataType)) { +dimensionValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL; + } +} +ByteBuffer buffer = ByteBuffer --- End diff -- move `ByteBuffer` to next line --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84242215 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/encodestep/EncoderProcessorStepImpl.java --- @@ -0,0 +1,63 @@ +/* + * 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.steps.encodestep; + +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep; +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.encoding.RowEncoder; +import org.apache.carbondata.processing.newflow.encoding.impl.RowEncoderImpl; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Encode data with dictionary values and composed with bit/byte packed key. + * nondictionary values are packed as bytes, and complex types are also packed as bytes. + */ +public class EncoderProcessorStepImpl extends AbstractDataLoadProcessorStep { --- End diff -- Actually this step is to transform the row into a temporarily encoded row (3 elements with dictionary dimension, no dictionary dimension, measure), the dimension byte array is what we want for sorting step, so can we call this step `SortKeyPreparationStep`? --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84240216 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/encoding/impl/RowEncoderImpl.java --- @@ -0,0 +1,132 @@ +/* + * 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.nio.ByteBuffer; +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.constants.IgnoreDictionary; +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.FieldEncoder; +import org.apache.carbondata.processing.newflow.encoding.RowEncoder; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; +import org.apache.carbondata.processing.util.RemoveDictionaryUtil; + +/** + * + */ +public class RowEncoderImpl implements RowEncoder { + + private CarbonDataLoadConfiguration configuration; + + private AbstractDictionaryFieldEncoderImpl[] dictionaryFieldEncoders; + + private NonDictionaryFieldEncoderImpl[] nonDictionaryFieldEncoders; + + private MeasureFieldEncoderImpl[] measureFieldEncoders; + + public RowEncoderImpl(DataField[] fields, CarbonDataLoadConfiguration configuration) { +this.configuration = configuration; +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cachecache = cacheProvider +.createCache(CacheType.REVERSE_DICTIONARY, +configuration.getTableIdentifier().getStorePath()); +List dictFieldEncoders = new ArrayList<>(); +List nonDictFieldEncoders = new ArrayList<>(); +List measureFieldEncoderList = new ArrayList<>(); + +long lruCacheStartTime = System.currentTimeMillis(); + +for (int i = 0; i < fields.length; i++) { + FieldEncoder fieldEncoder = FieldEncoderFactory.getInstance() + .createFieldEncoder(fields[i], cache, + configuration.getTableIdentifier().getCarbonTableIdentifier(), i); + if (fieldEncoder instanceof AbstractDictionaryFieldEncoderImpl) { +dictFieldEncoders.add((AbstractDictionaryFieldEncoderImpl) fieldEncoder); + } else if (fieldEncoder instanceof NonDictionaryFieldEncoderImpl) { +nonDictFieldEncoders.add((NonDictionaryFieldEncoderImpl) fieldEncoder); + } else if (fieldEncoder instanceof MeasureFieldEncoderImpl) { +measureFieldEncoderList.add((MeasureFieldEncoderImpl)fieldEncoder); + } +} +CarbonTimeStatisticsFactory.getLoadStatisticsInstance() +.recordLruCacheLoadTime((System.currentTimeMillis() - lruCacheStartTime) / 1000.0); +dictionaryFieldEncoders = +dictFieldEncoders.toArray(new AbstractDictionaryFieldEncoderImpl[dictFieldEncoders.size()]); +nonDictionaryFieldEncoders = nonDictFieldEncoders +.toArray(new NonDictionaryFieldEncoderImpl[nonDictFieldEncoders.size()]); +measureFieldEncoders = measureFieldEncoderList +.toArray(new MeasureFieldEncoderImpl[measureFieldEncoderList.size()]); + + } + + @Override public CarbonRow encode(CarbonRow row) throws CarbonDataLoadingException { --- End
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84238092 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/encoding/RowEncoder.java --- @@ -0,0 +1,32 @@ +/* + * 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; + +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Encodes the row + */ +public interface RowEncoder { --- End diff -- Calling this as Encoder is confusing, this step only replace dictionary value with dictionary key, right? It does not do actual encoding which is done in final write step Suggest change to `KeyedRowConverter`. See other comments in this PR. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
Github user jackylk commented on a diff in the pull request: https://github.com/apache/incubator-carbondata/pull/244#discussion_r84243728 --- Diff: processing/src/main/java/org/apache/carbondata/processing/newflow/steps/encodestep/EncoderProcessorStepImpl.java --- @@ -0,0 +1,63 @@ +/* + * 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.steps.encodestep; + +import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep; +import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration; +import org.apache.carbondata.processing.newflow.DataField; +import org.apache.carbondata.processing.newflow.encoding.RowEncoder; +import org.apache.carbondata.processing.newflow.encoding.impl.RowEncoderImpl; +import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException; +import org.apache.carbondata.processing.newflow.row.CarbonRow; + +/** + * Encode data with dictionary values and composed with bit/byte packed key. + * nondictionary values are packed as bytes, and complex types are also packed as bytes. + */ +public class EncoderProcessorStepImpl extends AbstractDataLoadProcessorStep { + + private RowEncoder encoder; + + public EncoderProcessorStepImpl(CarbonDataLoadConfiguration configuration, + AbstractDataLoadProcessorStep child) { +super(configuration, child); + } + + @Override public DataField[] getOutput() { +return child.getOutput(); + } + + @Override public void intialize() throws CarbonDataLoadingException { +encoder = new RowEncoderImpl(child.getOutput(), configuration); +child.intialize(); + } + + @Override protected CarbonRow processRow(CarbonRow row) { +return encoder.encode(row); + } + + @Override public void finish() { +encoder.finish(); + } + + @Override public void close() { + --- End diff -- in case of failure, should there be encode.close()? --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---
[GitHub] incubator-carbondata pull request #244: [CARBONDATA-300] Added Encoder proce...
GitHub user ravipesala opened a pull request: https://github.com/apache/incubator-carbondata/pull/244 [CARBONDATA-300] Added Encoder processor for dataloading. Added interface implementation for encode data load processor. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ravipesala/incubator-carbondata encode-step Alternatively you can review and apply these changes as the patch at: https://github.com/apache/incubator-carbondata/pull/244.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #244 commit e6eb5d1137bd82beed642e7206ebb566dcb81fa3 Author: ravipesalaDate: 2016-10-17T05:03:35Z Added Encoder processor for dataloding. --- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---