[GitHub] carbondata pull request #2607: [CARBONDATA-2818] Presto Upgrade to 0.206
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2607#discussion_r207804289 --- Diff: integration/presto/src/main/scala/org/apache/carbondata/presto/CarbonDictionaryDecodeReadSupport.scala --- @@ -84,25 +85,31 @@ class CarbonDictionaryDecodeReadSupport[T] extends CarbonReadSupport[T] { * @param dictionaryData * @return */ - private def createSliceArrayBlock(dictionaryData: Dictionary): SliceArrayBlock = { + private def createSliceArrayBlock(dictionaryData: Dictionary): Block = { val chunks: DictionaryChunksWrapper = dictionaryData.getDictionaryChunks -val sliceArray = new Array[Slice](chunks.getSize + 1) -// Initialize Slice Array with Empty Slice as per Presto's code -sliceArray(0) = Slices.EMPTY_SLICE -var count = 1 +val positionCount = chunks.getSize; +val offsetVector : Array[Int] = new Array[Int](positionCount + 2 ) +val isNullVector: Array[Boolean] = new Array[Boolean](positionCount + 1) +isNullVector(0) = true +isNullVector(1) = true --- End diff -- We are talking about dictionary here , so In dictionary there will be only one null and the key value will be 1 by default in CarbonData, hence the isNullVector will be populated only once with null value it has no bearing on actual data. The Carbondata key starts from 1 so we need a filler at 0th position and 1 index is actually Null to map to carbondata null values . The offset index will be like 0th Position -> 0 (As it is filler) 1st Position -> 0 (For actual Null) 2nd Postion -> 0 as the byte[] is still null so starting point will be 0 only ---
[GitHub] carbondata pull request #2412: [CARBONDATA-2656] Presto vector stream reader...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2412#discussion_r198136493 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java --- @@ -166,61 +146,31 @@ protected void closeWithSuppression(Throwable throwable) /** * Lazy Block Implementation for the Carbondata */ - private final class CarbondataBlockLoader - implements LazyBlockLoader - { + private final class CarbondataBlockLoader implements LazyBlockLoader { private final int expectedBatchId = batchId; private final int columnIndex; -private final Type type; private boolean loaded; -public CarbondataBlockLoader(int columnIndex, Type type) -{ +CarbondataBlockLoader(int columnIndex) { this.columnIndex = columnIndex; - this.type = requireNonNull(type, "type is null"); } -@Override -public final void load(LazyBlock lazyBlock) -{ +@Override public final void load(LazyBlock lazyBlock) { if (loaded) { return; } checkState(batchId == expectedBatchId); try { -Block block = readers[columnIndex].readBlock(type); +Block block = +((PrestoVectorBlockBuilder) vectorReader.getColumnarBatch().column(columnIndex)) +.buildBlock(lazyBlock.getPositionCount()); --- End diff -- You should be setting the batch size only, no need for two fields ---
[GitHub] carbondata pull request #2412: [CARBONDATA-2656] Presto vector stream reader...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2412#discussion_r198136861 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/readers/ObjectStreamReader.java --- @@ -17,50 +17,50 @@ package org.apache.carbondata.presto.readers; -import java.io.IOException; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl; import com.facebook.presto.spi.block.Block; import com.facebook.presto.spi.block.BlockBuilder; import com.facebook.presto.spi.block.BlockBuilderStatus; +import com.facebook.presto.spi.type.IntegerType; import com.facebook.presto.spi.type.Type; /** * Class to read the Object Stream */ -public class ObjectStreamReader extends AbstractStreamReader { +public class ObjectStreamReader extends CarbonColumnVectorImpl implements PrestoVectorBlockBuilder { + protected int batchSize; + protected Type type = IntegerType.INTEGER; - public ObjectStreamReader() { + protected BlockBuilder builder; + public ObjectStreamReader(int batchSize, DataType dataType) { +super(batchSize, dataType); +this.batchSize = batchSize; +this.builder = type.createBlockBuilder(new BlockBuilderStatus(), batchSize); } - /** - * Function to create the object Block - * @param type - * @return - * @throws IOException - */ - public Block readBlock(Type type) throws IOException { -int numberOfRows = 0; -BlockBuilder builder = null; -if (isVectorReader) { - numberOfRows = batchSize; - builder = type.createBlockBuilder(new BlockBuilderStatus(), numberOfRows); - if (columnVector != null) { -for (int i = 0; i < numberOfRows; i++) { - type.writeObject(builder, columnVector.getData(i)); -} - } -} else { - numberOfRows = streamData.length; - builder = type.createBlockBuilder(new BlockBuilderStatus(), numberOfRows); - for (int i = 0; i < numberOfRows; i++) { -type.writeObject(builder, streamData[i]); - } -} - + @Override public Block buildBlock(int blockSize) { --- End diff -- remove the parameter blockSize ---
[GitHub] carbondata pull request #2412: [CARBONDATA-2656] Presto vector stream reader...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2412#discussion_r198107936 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java --- @@ -20,50 +20,81 @@ import java.util.HashSet; import java.util.Set; +import org.apache.carbondata.core.cache.dictionary.Dictionary; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.datatype.DataTypes; +import org.apache.carbondata.core.metadata.datatype.DecimalType; import org.apache.carbondata.core.metadata.datatype.StructField; import org.apache.carbondata.core.scan.result.vector.impl.CarbonColumnVectorImpl; +import org.apache.carbondata.presto.readers.BooleanStreamReader; +import org.apache.carbondata.presto.readers.DecimalSliceStreamReader; +import org.apache.carbondata.presto.readers.DoubleStreamReader; +import org.apache.carbondata.presto.readers.IntegerStreamReader; +import org.apache.carbondata.presto.readers.LongStreamReader; +import org.apache.carbondata.presto.readers.ObjectStreamReader; +import org.apache.carbondata.presto.readers.ShortStreamReader; +import org.apache.carbondata.presto.readers.SliceStreamReader; +import org.apache.carbondata.presto.readers.TimestampStreamReader; + +import com.facebook.presto.spi.block.SliceArrayBlock; public class CarbonVectorBatch { - private static final int DEFAULT_BATCH_SIZE = 4 * 1024; + private static final int DEFAULT_BATCH_SIZE = 4 * 1024; - private final StructField[] schema; private final int capacity; - private int numRows; private final CarbonColumnVectorImpl[] columns; - // True if the row is filtered. private final boolean[] filteredRows; - // Column indices that cannot have null values. private final Set nullFilteredColumns; - + private int numRows; // Total number of rows that have been filtered. private int numRowsFiltered = 0; - - private CarbonVectorBatch(StructField[] schema, int maxRows) { -this.schema = schema; + private CarbonVectorBatch(StructField[] schema, CarbonDictionaryDecodeReadSupport readSupport, + int maxRows) { this.capacity = maxRows; this.columns = new CarbonColumnVectorImpl[schema.length]; this.nullFilteredColumns = new HashSet<>(); this.filteredRows = new boolean[maxRows]; +Dictionary[] dictionaries = readSupport.getDictionaries(); +DataType[] dataTypes = readSupport.getDataTypes(); for (int i = 0; i < schema.length; ++i) { - StructField field = schema[i]; - columns[i] = new CarbonColumnVectorImpl(maxRows, field.getDataType()); + columns[i] = createDirectStreamReader(maxRows, dataTypes[i], schema[i], dictionaries[i], + readSupport.getSliceArrayBlock(i)); --- End diff -- To be consistent can you get the sliceArrayBlock and pass it on as a parameter. ---
[GitHub] carbondata pull request #2265: Added Performance Optimization for Presto by ...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2265#discussion_r196799725 --- Diff: integration/presto/README.md --- @@ -113,6 +116,10 @@ Please follow the below steps to query carbondata in presto enable.unsafe.in.query.processing property by default is true in CarbonData system, the carbon.unsafe.working.memory.in.mb property defines the limit for Unsafe Memory usage in Mega Bytes, the default value is 512 MB. If your tables are big you can increase the unsafe memory, or disable unsafe via setting enable.unsafe.in.query.processing=false. + + If you do not want to use unsafe memory at all please set the below properties to false as well. --- End diff -- This has been corrected ---
[GitHub] carbondata pull request #2265: Added Performance Optimization for Presto by ...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2265#discussion_r196799800 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java --- @@ -152,19 +176,20 @@ private ConnectorTableMetadata getTableMetadata(SchemaTableName schemaTableName) Type spiType = carbonDataType2SpiMapper(cs); columnHandles.put(cs.getColumnName(), - new CarbondataColumnHandle(connectorId, cs.getColumnName(), spiType, column.getSchemaOrdinal(), - column.getKeyOrdinal(), column.getColumnGroupOrdinal(), false, cs.getColumnGroupId(), - cs.getColumnUniqueId(), cs.isUseInvertedIndex(), cs.getPrecision(), cs.getScale())); + new CarbondataColumnHandle(connectorId, cs.getColumnName(), spiType, + column.getSchemaOrdinal(), column.getKeyOrdinal(), column.getColumnGroupOrdinal(), + false, cs.getColumnGroupId(), cs.getColumnUniqueId(), cs.isUseInvertedIndex(), + cs.getPrecision(), cs.getScale())); } for (CarbonMeasure measure : cb.getMeasureByTableName(tableName)) { ColumnSchema cs = measure.getColumnSchema(); - Type spiType = carbonDataType2SpiMapper(cs); columnHandles.put(cs.getColumnName(), - new CarbondataColumnHandle(connectorId, cs.getColumnName(), spiType, cs.getSchemaOrdinal(), - measure.getOrdinal(), cs.getColumnGroupId(), true, cs.getColumnGroupId(), - cs.getColumnUniqueId(), cs.isUseInvertedIndex(), cs.getPrecision(), cs.getScale())); + new CarbondataColumnHandle(connectorId, cs.getColumnName(), spiType, + cs.getSchemaOrdinal(), measure.getOrdinal(), cs.getColumnGroupId(), true, + cs.getColumnGroupId(), cs.getColumnUniqueId(), cs.isUseInvertedIndex(), + cs.getPrecision(), cs.getScale())); } //should i cache it? --- End diff -- Removed the comment ---
[GitHub] carbondata pull request #2265: Added Performance Optimization for Presto by ...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2265#discussion_r196799513 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSourceProvider.java --- @@ -129,23 +135,31 @@ private QueryModel createQueryModel(CarbondataSplit carbondataSplit, String carbonTablePath = carbonTable.getAbsoluteTableIdentifier().getTablePath(); conf.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath); + conf.set("query.id", queryId); JobConf jobConf = new JobConf(conf); CarbonTableInputFormat carbonTableInputFormat = createInputFormat(jobConf, carbonTable, PrestoFilterUtil.parseFilterExpression(carbondataSplit.getConstraints()), carbonProjection); TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(jobConf, new TaskAttemptID("", 1, TaskType.MAP, 0, 0)); - CarbonInputSplit carbonInputSplit = - CarbonLocalInputSplit.convertSplit(carbondataSplit.getLocalInputSplit()); + CarbonMultiBlockSplit carbonInputSplit = + CarbonLocalMultiBlockSplit.convertSplit(carbondataSplit.getLocalInputSplit()); QueryModel queryModel = carbonTableInputFormat.createQueryModel(carbonInputSplit, hadoopAttemptContext); + queryModel.setQueryId(queryId); queryModel.setVectorReader(true); + queryModel.setStatisticsRecorder( + CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId())); + /* List splitList = new ArrayList<>(1); - splitList.add(carbonInputSplit); - List tableBlockInfoList = CarbonInputSplit.createBlocks(splitList); --- End diff -- This has been fixed ---
[GitHub] carbondata issue #2265: Added Performance Optimization for Presto by using M...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/2265 retest this please ---
[GitHub] carbondata issue #2265: Added Performance Optimization for Presto by using M...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/2265 retest this please ---
[GitHub] carbondata issue #2265: Added Performance Optimization for Presto by using M...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/2265 Yes we have created a JIRA for the same CARBONDATA-2583 ---
[GitHub] carbondata pull request #2340: [CARBONDATA-2529] Fixed S3 Issue for Hadoop 2...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/2340 [CARBONDATA-2529] Fixed S3 Issue for Hadoop 2.8.3 This issue fixes the issue while loading the data with S3 as backend. - [ N] Any interfaces changed? - [N ] Any backward compatibility impacted? - [ N] Document update required? - [Y ] Testing done You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-2529 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/2340.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 #2340 commit a05a7db79c5d1e25fc78b65f07b5f5fedc8611de Author: Bhavya <bhavya@...> Date: 2018-05-24T15:47:58Z Fixed S3 Issue for Hadoop 2.8.3 ---
[GitHub] carbondata pull request #2265: Added Performance Optimization for Presto by ...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2265#discussion_r189779766 --- Diff: integration/presto/pom.xml --- @@ -462,12 +462,6 @@ 3.0.2 --- End diff -- @chenliang613 we can not remove this dependency as CarbonTableInputFormat is part of this , we have excluded a lot of dependencies which were not needed in this particular dependency. ---
[GitHub] carbondata issue #2265: Added Performance Optimization for Presto by using M...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/2265 I have resolved the conflicts ---
[GitHub] carbondata pull request #2265: Added Performance Optimization for Presto by ...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2265#discussion_r186359964 --- Diff: integration/presto/pom.xml --- @@ -462,12 +462,6 @@ 3.0.2 --- End diff -- We can not remove this class for now as compilation fails , is there a particular reason we should have not this dependency then I can look into supplementing it with some carbon jar ---
[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2139#discussion_r179913482 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java --- @@ -415,14 +440,56 @@ public TBase create() { return result; } + /** Returns list of partition specs to query based on the domain constraints + * @param constraints + * @param carbonTable + * @throws IOException + */ + private List findRequiredPartitions(TupleDomain constraints, CarbonTable carbonTable, + LoadMetadataDetails[]loadMetadataDetails) { +Set partitionSpecs = new HashSet<>(); +List prunePartitions = new ArrayList(); + +for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) { + SegmentFileStore segmentFileStore = null; + try { +segmentFileStore = +new SegmentFileStore(carbonTable.getTablePath(), loadMetadataDetail.getSegmentFile()); +partitionSpecs.addAll(segmentFileStore.getPartitionSpecs()); + + } catch (IOException e) { +e.printStackTrace(); + } +} + +List partitionValuesFromExpression = +PrestoFilterUtil.getPartitionFilters(carbonTable, constraints); + +List<List> partitionSpecNamesList = partitionSpecs.stream().map( +PartitionSpec::getPartitions).collect(Collectors.toList()); + +List partitionSpecsList = new ArrayList(partitionSpecs); + +for (int i = 0; i < partitionSpecNamesList.size(); i++) { --- End diff -- I am not sure about this logic , you have a set and you create a specsNameList and then you create a list of partitionSpecs and then you check that if the list has all fiteredPartition you add it to prunedPartitiion, why not do it in the 1st map itself instead of complicating code ---
[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2139#discussion_r179913431 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java --- @@ -415,14 +440,56 @@ public TBase create() { return result; } + /** Returns list of partition specs to query based on the domain constraints + * @param constraints + * @param carbonTable + * @throws IOException + */ + private List findRequiredPartitions(TupleDomain constraints, CarbonTable carbonTable, + LoadMetadataDetails[]loadMetadataDetails) { +Set partitionSpecs = new HashSet<>(); +List prunePartitions = new ArrayList(); + +for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) { + SegmentFileStore segmentFileStore = null; + try { +segmentFileStore = +new SegmentFileStore(carbonTable.getTablePath(), loadMetadataDetail.getSegmentFile()); +partitionSpecs.addAll(segmentFileStore.getPartitionSpecs()); + + } catch (IOException e) { +e.printStackTrace(); + } +} + +List partitionValuesFromExpression = +PrestoFilterUtil.getPartitionFilters(carbonTable, constraints); + +List<List> partitionSpecNamesList = partitionSpecs.stream().map( +PartitionSpec::getPartitions).collect(Collectors.toList()); + +List partitionSpecsList = new ArrayList(partitionSpecs); + +for (int i = 0; i < partitionSpecNamesList.size(); i++) { + List partitionSpecNames = partitionSpecNamesList.get(i); + if (partitionSpecNames.containsAll(partitionValuesFromExpression)) { +prunePartitions +.add(partitionSpecsList.get(i)); + } +} +return prunePartitions; + } + private CarbonTableInputFormat createInputFormat( Configuration conf, - AbsoluteTableIdentifier identifier, Expression filterExpression) - throws IOException { + AbsoluteTableIdentifier identifier, Expression filterExpression, List filteredPartitions) + throws IOException { CarbonTableInputFormat format = new CarbonTableInputFormat(); CarbonTableInputFormat.setTablePath(conf, -identifier.appendWithLocalPrefix(identifier.getTablePath())); +identifier.appendWithLocalPrefix(identifier.getTablePath())); CarbonTableInputFormat.setFilterPredicates(conf, filterExpression); - +if(filteredPartitions.size() != 0) { + CarbonTableInputFormat.setPartitionsToPrune(conf, new ArrayList<>(filteredPartitions)); --- End diff -- Why Create a new ArrayList here when the calling method is already returning a List ---
[GitHub] carbondata pull request #2139: [CARBONDATA-2267] [Presto] Support Reading Ca...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2139#discussion_r179913191 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java --- @@ -387,11 +398,25 @@ public TBase create() { config.set(CarbonTableInputFormat.DATABASE_NAME, carbonTable.getDatabaseName()); config.set(CarbonTableInputFormat.TABLE_NAME, carbonTable.getTableName()); +JobConf jobConf = new JobConf(config); +List filteredPartitions = new ArrayList(); + +try { + loadMetadataDetails= SegmentStatusManager + .readTableStatusFile(CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath())); +} catch (IOException e) { + e.printStackTrace(); --- End diff -- The Error should be logged and if there is an IOException than it needs to be handled or thrown to make sure that error is propogated. ---
[GitHub] carbondata issue #1940: [CARBONDATA-2140 ] Refactoring code to improve perfo...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1940 This PR Removes the following classes from the PrestoCode -CarbondataRecordCursor - This class used to get the records row by row from the iterator, we were using this just as a placeholder for values so this class was not needed. -CarbondataRecordSet - This class was used to create the RecordCursor and since we were not using the Row by Row reader this class is also removed -CarbondataRecordSetProvider - All the functionality of this class has been modularized and move to CarbondataPageSourceProvider. - CarbondataUtil - This class was not used in the actual code. ---
[GitHub] carbondata pull request #2114: [CARBONDATA-2295] Added property for UnsafeMe...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/2114#discussion_r178230596 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java --- @@ -366,7 +368,10 @@ public TBase create() { public List getInputSplits2(CarbonTableCacheModel tableCacheModel, Expression filters) { List result = new ArrayList<>(); - +if(config.getUnsafeMemoryInMb() != null) { + CarbonProperties.getInstance().addProperty(CarbonCommonConstants.UNSAFE_WORKING_MEMORY_IN_MB, --- End diff -- Done ---
[GitHub] carbondata pull request #2114: [CARBONDATA-2295] Added property for UnsafeMe...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/2114 [CARBONDATA-2295] Added property for UnsafeMemory configuration Added property for setting up the unsafe memory limit in MB - [ X] Testing done Added a new property in the test cases to make sure they all run fine Tested manually on the presto client as well You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-2295 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/2114.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 #2114 commit 9e44537b97340e71829f733d878e1dfab8affbdc Author: Bhavya <bhavya@...> Date: 2018-03-29T15:03:34Z Added property for UnsafeMemory configuration ---
[GitHub] carbondata issue #1940: [CARBONDATA-2140 ] Refactoring code to improve perfo...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1940 retest sdv please. ---
[GitHub] carbondata pull request #1940: [CARBONDATA-2140 ] Refactoring code to improv...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1940 [CARBONDATA-2140 ] Refactoring code to improve performance and removing unnecessary code in Presto Integration This PR is for optimizing Presto performance and refactoring the code to remove unnecessary classes and making it simpler. Be sure to do all of the following checklist to help us incorporate your contribution quickly and easily: - [X] Any interfaces changed? No - [X] Any backward compatibility impacted? No - [X] Documentation Remains Same - [X] Testing done - No new test cases needed as the functionality remain same - All existing test cases are passing. - Ran the TPCH queries to check the performance . - [X] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-2140 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1940.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 #1940 commit 533c1d2db0d4608081ed65bb98ea3bbe94e189ea Author: Bhavya <bhavya@...> Date: 2018-02-06T13:27:42Z Reafactored Code to remove Cursor and RecordSetProvider ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r156023011 --- Diff: core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java --- @@ -238,7 +238,7 @@ private void fillDataForNonExistingMeasures() { (long) defaultValue); } else if (DataTypes.isDecimal(dataType)) { vector.putDecimals(columnVectorInfo.vectorOffset, columnVectorInfo.size, -(Decimal) defaultValue, measure.getPrecision()); +((Decimal) defaultValue).toJavaBigDecimal(), measure.getPrecision()); --- End diff -- We can not remove these imports as the data that is returned is stored in Spark Format , the casting is done to convert it to generic types ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r155492390 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataConnector.java --- @@ -31,13 +31,13 @@ private static final Logger log = Logger.get(CarbondataConnector.class); private final LifeCycleManager lifeCycleManager; - private final CarbondataMetadata metadata; + private final ConnectorMetadata metadata; --- End diff -- CarbonDataMetadata extends ConnectorMetaData, we should always code to Interface hence have changed this, also this way in concurrency we are using the standard Presto protocol and providing a ClassLoaderSafeConnectorMetadata ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r155438974 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorBatch.java --- @@ -0,0 +1,112 @@ +package org.apache.carbondata.presto; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +public class CarbonVectorBatch { + + private static final int DEFAULT_BATCH_SIZE = 1024; --- End diff -- The batchSize is the size of the complete batch so for every Column we have to set it as byteSize , it is the number of records in the batch so it has to be equivalent to the batch size ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r155435994 --- Diff: integration/presto/pom.xml --- @@ -431,14 +435,27 @@ org.apache.spark - spark-sql_2.11 + spark-network-common_2.11 --- End diff -- Yes it is used only in Tests, we have defined the scope as test for all these dependencies ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r155185364 --- Diff: integration/presto/pom.xml --- @@ -431,14 +435,27 @@ org.apache.spark - spark-sql_2.11 + spark-network-common_2.11 --- End diff -- For Running the Integration Tests as when we create a CarbonStore we need the Spark Dependency, so what we have done is that we have kept the scope of those dependency as Test , no spark Jar will be part of the final Jar that we copy to presto plugin ---
[GitHub] carbondata pull request #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1581#discussion_r154860147 --- Diff: integration/presto/pom.xml --- @@ -31,7 +31,7 @@ presto-plugin -0.186 +0.187 --- End diff -- Yes I think we need to specify 0.187 version upgrade in another mailing list, with 0.187 in carbon it works with 0.186 but not with 0.166 as the interfaces for Connector is changed in 0.186 ---
[GitHub] carbondata issue #1581: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1581 retest this please. ---
[GitHub] carbondata issue #1538: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1538 This PR removes the Spark Dependency from Presto Integration Module for using the CarbonVectorizedRecordreader, This PR consolidate CarbonVectorizedRecordReader into one,to make it shared for all integration modules In the earlier version of Presto Integration we were using ColumnarBatch of Spark, which is not a good practice, here we provided our own implementation of the ColumnVector and the VectorBatch to eliminate the Spark all together. This generic ColumnVector can now be used for all the integration module wherever we want to have a VectorizedReader to speed up the processing. There are some core module classes changed to ensure that we are using Java data types instead of Spark datatypes, Decimal being one of them. This PR tries to limit the changes to Core module . Newly Added Classes 1.CarbonColumnVectorImpl:This Class Implements the Interface CarbonColumnVector and provides the methods to store the data in a Vector and to retrieved the data from it as well 2.CarbonVectorBatch: This Class Creates A VectorizedRowBatch which is a set of rows, organized with each column as a CarbonVector. It is the unit of query execution, organized to minimize the cost per row and achieve high cycles-per-instruction. The major fields are public by design to allow fast and convenient access by the vectorized query execution code. 3.StructField:This class is used to pass the Schema Information to the Carbon Columnar Batch ---
[GitHub] carbondata pull request #1538: [CARBONDATA-1779] GenericVectorizedReader
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1538#discussion_r152498613 --- Diff: integration/presto/pom.xml --- @@ -31,7 +31,7 @@ presto-plugin -0.186 --- End diff -- There was an issue with multiple queries on Presto 0.186 so they have declare that release as unstable, they fixed that issue in 0.187 so have upgraded it to 0.187 ---
[GitHub] carbondata pull request #1538: [CARBONDATA-1779] GenericVectorizedReader
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1538 [CARBONDATA-1779] GenericVectorizedReader Be sure to do all of the following checklist to help us incorporate your contribution quickly and easily: - No interfaces changed? - No backward compatibility impacted? - No Document update required? - [ Yes] Testing done - All Unit test cases are passing, no new unit test cases were needed as this PR implements a Generic Vectorized Reader. - Manual Testing completed for the same . You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-1779 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1538.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 #1538 commit ef28391c656cc2d20082e52dd4ab729b0992cfb3 Author: Bhavya <bha...@knoldus.com> Date: 2017-11-14T10:05:44Z Added Generic vectorized Reader ---
[GitHub] carbondata issue #1477: [CARBONDATA-1686] Presto Version Upgrade to 0.186
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1477 retest this please ---
[GitHub] carbondata pull request #1477: [CARBONDATA-1686] Presto Version Upgrade to 0...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1477#discussion_r149945794 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/PrestoFilterUtil.java --- @@ -261,8 +261,6 @@ else if (type instanceof DecimalType) { return new BigDecimal(new BigInteger(String.valueOf(rawdata)), ((DecimalType) type).getScale()); } -} else if (type.equals(TimestampType.TIMESTAMP)) { --- End diff -- This has been fixed ---
[GitHub] carbondata issue #1477: [CARBONDATA-1686] Presto Version Upgrade to 0.186
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1477 All issues have been resolved ---
[GitHub] carbondata pull request #1477: [CARBONDATA-1686] Presto Version Upgrade to 0...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1477#discussion_r149944113 --- Diff: integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java --- @@ -78,10 +78,6 @@ private CarbondataPageSource(List types, RecordCursor cursor) { this.readers = createStreamReaders(); } - @Override public long getTotalBytes() { --- End diff -- The Presto 0.186 has remove this method from specification, so we have to remove it from our code as well ---
[GitHub] carbondata pull request #1477: [CARBONDATA-1686] Presto Version Upgrade to 0...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1477 [CARBONDATA-1686] Presto Version Upgrade to 0.186 - [No] Any interfaces changed? - [Yes ] Any backward compatibility impacted? Carbondata will now work with Presto Server 0.186 onward - [ No] Document update required? - [ Yes] Testing done Since it is a version upgrade all the integration test cases are executed successfuly. Performance testing done manually and there is improvement in query execution times. You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata PrestoUpgrade Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1477.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 #1477 commit d2c8a479f0491b35b13ad6cf79595188ac609deb Author: Bhavya <bha...@knoldus.com> Date: 2017-11-09T07:25:50Z Presto Version upgraded to 0.186 ---
[GitHub] carbondata issue #1348: [CARBONDATA-1469] Optimizations for Presto Integrati...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1348 retest this please ---
[GitHub] carbondata pull request #1348: [CARBONDATA-1469] Optimizations for Presto In...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1348 [CARBONDATA-1469] Optimizations for Presto Integration This PR has added following new features - Optimized Stream Readers to handle Nulls in vector efficiently - Added TimestampStreamReader and ShortStreamReader to handle Timestamp and Short datatype correctly. - Modified the Filter code for efficient filter pushdown to Carbondata. - Optimize the POM to reduce the dependency size. - Verified that Build is passing for 2.1 - All test cases are passing - Manually verified some queries to make sure everything is working fine. You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-1469 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1348.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 #1348 commit f49e42e5475bd5e73857488d6ca0d2aacbef656e Author: Bhavya <bha...@knoldus.com> Date: 2017-09-11T11:03:07Z Optimizations in Presto Code ---
[GitHub] carbondata pull request #1307: [CARBONDATA-1433] Added Vectorized Reader for...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1307 [CARBONDATA-1433] Added Vectorized Reader for Presto Integration This PR is for optimizing the Presto Integration Performance - Added Vectorized Reader for reading the data - Used DictionaryBlock for loading the dictionary values. - Removed unused code - Build is Successful - Verified that all test cases are passing - Manually Verified by running queries You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-1433 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1307.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 #1307 commit 8e8f9ad5335365632f636622afe9fe1565086a90 Author: Bhavya <bha...@knoldus.com> Date: 2017-08-29T11:32:18Z Added Vectorized Reader for Presto Integration --- 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] carbondata pull request #1294: [CARBONDATA-1418] Use CarbonTableInputFormat ...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1294 [CARBONDATA-1418] Use CarbonTableInputFormat for creating the Splits and QueryModel Refactored Code to use CarbonTableInputFormat for creating splits Build Successful All Test Cases executing successfully Manually verified the changes You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata prestoImprovement Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1294.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 #1294 commit 86c7d2240555b2579726a9f6378f51af93ab2677 Author: Bhavya <bha...@knoldus.com> Date: 2017-08-28T17:51:58Z Used CarbonTableInputFormat for creating the Splits and QueryModel --- 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] carbondata issue #1236: fixed bug for fetching the error value of decimal ty...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1236 Hi @steven-qin can you please remove the extra import " import com.google.common.base.Strings;" from CarbondataRecordCursor. I have verified the PR for both LongDecimalType and ShortDecimalType and it is working fine. --- 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] carbondata pull request #1190: [CARBONDATA-1323] Presto Optimization for Int...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1190#discussion_r129588108 --- Diff: integration/presto/pom.xml --- @@ -228,6 +228,33 @@ true + + org.scala-tools + maven-scala-plugin --- End diff -- I have written the dictionary decoding in scala as it is more optimized and easier to understand, hence we have to add this plugin for compiling the scala code --- 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] carbondata issue #1142: [CARBONDATA-1271] Enhanced Performance for Hive Inte...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/1142 @cenyuhai The performance improved a lot I tested it with 5 Million records please see the attached results [Performance.txt](https://github.com/apache/carbondata/files/1135276/Performance.txt) --- 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] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1142#discussion_r126392840 --- Diff: integration/hive/src/main/java/org/apache/carbondata/hive/DictionaryDecodeReadSupport.java --- @@ -0,0 +1,288 @@ +/* + * 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.hive; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; +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.metadata.AbsoluteTableIdentifier; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.encoder.Encoding; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; +import org.apache.carbondata.core.util.CarbonUtil; + +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; + +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.GenericArrayData; + +/** + * This is the class to decode dictionary encoded column data back to its original value. + */ +public class DictionaryDecodeReadSupport implements CarbonReadSupport { --- End diff -- Will change the file name to CarbonDictionaryDecodeReadSupport. --- 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] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1142#discussion_r126392125 --- Diff: hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java --- @@ -444,9 +444,14 @@ protected Expression getFilterPredicates(Configuration configuration) { } } } + + // For Hive integration if we have to get the stats we have to fetch hive.query.id + String query_id = job.getConfiguration().get("query.id") != null ? + job.getConfiguration().get("query.id") : + job.getConfiguration().get("hive.query.id"); --- End diff -- It is set in the configuration internally by Hive --- 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] carbondata issue #984: [CARBONDATA-1008] Make Caron table schema compatible ...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/984 Sure Liang, I will review it today. Regards Bhavya On Sun, Jul 9, 2017 at 7:39 PM, å²çæµ· <notificati...@github.com> wrote: > @chenliang613 <https://github.com/chenliang613> can you review this pr? > > â > You are receiving this because you commented. > Reply to this email directly, view it on GitHub > <https://github.com/apache/carbondata/pull/984#issuecomment-313922080>, > or mute the thread > <https://github.com/notifications/unsubscribe-auth/ADlpPikxNdwpDsB4r1_GOEKaNYdUuwMJks5sMN8EgaJpZM4Nrie4> > . > --- 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] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1142#discussion_r126119484 --- Diff: integration/hive/src/main/java/org/apache/carbondata/hive/DictionaryDecodeReadSupport.java --- @@ -0,0 +1,288 @@ +/* + * 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.hive; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; +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.metadata.AbsoluteTableIdentifier; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.encoder.Encoding; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; +import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; + +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.GenericArrayData; + +/** + * This is the class to decode dictionary encoded column data back to its original value. + */ +public class DictionaryDecodeReadSupport implements CarbonReadSupport { + + protected Dictionary[] dictionaries; + + protected DataType[] dataTypes; + /** + * carbon columns + */ + protected CarbonColumn[] carbonColumns; + + protected Writable[] writableArr; + + /** + * This initialization is done inside executor task + * for column dictionary involved in decoding. + * + * @param carbonColumns column list + * @param absoluteTableIdentifier table identifier + */ + @Override public void initialize(CarbonColumn[] carbonColumns, + AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException { +this.carbonColumns = carbonColumns; +dictionaries = new Dictionary[carbonColumns.length]; +dataTypes = new DataType[carbonColumns.length]; +for (int i = 0; i < carbonColumns.length; i++) { + if (carbonColumns[i].hasEncoding(Encoding.DICTIONARY) && !carbonColumns[i] + .hasEncoding(Encoding.DIRECT_DICTIONARY) && !carbonColumns[i].isComplex()) { +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider +.createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath()); +dataTypes[i] = carbonColumns[i].getDataType(); +dictionaries[i] = forwardDictionaryCache.get( +new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(), +carbonColumns[i].getColumnIdentifier(), dataTypes[i])); + } else { +dataTypes[i] = carbonColumns[i].getDataType(); + } +} + } + + @Override public T readRow(Object[] data) { +
[GitHub] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1142#discussion_r126119465 --- Diff: integration/hive/src/main/java/org/apache/carbondata/hive/DictionaryDecodeReadSupport.java --- @@ -0,0 +1,288 @@ +/* + * 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.hive; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; +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.metadata.AbsoluteTableIdentifier; +import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.metadata.encoder.Encoding; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; +import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; +import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.DoubleWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.hive.serde2.io.ShortWritable; +import org.apache.hadoop.hive.serde2.io.TimestampWritable; + +import org.apache.hadoop.io.ArrayWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.GenericArrayData; + +/** + * This is the class to decode dictionary encoded column data back to its original value. + */ +public class DictionaryDecodeReadSupport implements CarbonReadSupport { + + protected Dictionary[] dictionaries; + + protected DataType[] dataTypes; + /** + * carbon columns + */ + protected CarbonColumn[] carbonColumns; + + protected Writable[] writableArr; + + /** + * This initialization is done inside executor task + * for column dictionary involved in decoding. + * + * @param carbonColumns column list + * @param absoluteTableIdentifier table identifier + */ + @Override public void initialize(CarbonColumn[] carbonColumns, + AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException { +this.carbonColumns = carbonColumns; +dictionaries = new Dictionary[carbonColumns.length]; +dataTypes = new DataType[carbonColumns.length]; +for (int i = 0; i < carbonColumns.length; i++) { + if (carbonColumns[i].hasEncoding(Encoding.DICTIONARY) && !carbonColumns[i] + .hasEncoding(Encoding.DIRECT_DICTIONARY) && !carbonColumns[i].isComplex()) { +CacheProvider cacheProvider = CacheProvider.getInstance(); +Cache<DictionaryColumnUniqueIdentifier, Dictionary> forwardDictionaryCache = cacheProvider +.createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath()); +dataTypes[i] = carbonColumns[i].getDataType(); +dictionaries[i] = forwardDictionaryCache.get( +new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(), +carbonColumns[i].getColumnIdentifier(), dataTypes[i])); + } else { +dataTypes[i] = carbonColumns[i].getDataType(); + } +} + } + + @Override public T readRow(Object[] data) { +
[GitHub] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/1142#discussion_r126077264 --- Diff: core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsRecorderImpl.java --- @@ -101,45 +101,47 @@ public String collectExecutorStatistics() { long scannedPages = 0; try { for (QueryStatistic statistic : queryStatistics) { -switch (statistic.getMessage()) { - case QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR: -load_blocks_time += statistic.getTimeTaken(); -break; - case QueryStatisticsConstants.SCAN_BLOCKlET_TIME: -scan_blocks_time += statistic.getCount(); -break; - case QueryStatisticsConstants.SCAN_BLOCKS_NUM: -scan_blocks_num += statistic.getCount(); -break; - case QueryStatisticsConstants.LOAD_DICTIONARY: -load_dictionary_time += statistic.getTimeTaken(); -break; - case QueryStatisticsConstants.RESULT_SIZE: -result_size += statistic.getCount(); -break; - case QueryStatisticsConstants.EXECUTOR_PART: -total_executor_time += statistic.getTimeTaken(); -break; - case QueryStatisticsConstants.TOTAL_BLOCKLET_NUM: -total_blocklet = statistic.getCount(); -break; - case QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM: -valid_scan_blocklet = statistic.getCount(); -break; - case QueryStatisticsConstants.VALID_PAGE_SCANNED: -valid_pages_blocklet = statistic.getCount(); -break; - case QueryStatisticsConstants.TOTAL_PAGE_SCANNED: -total_pages = statistic.getCount(); -break; - case QueryStatisticsConstants.READ_BLOCKlET_TIME: -readTime = statistic.getCount(); -break; - case QueryStatisticsConstants.PAGE_SCANNED: -scannedPages = statistic.getCount(); -break; - default: -break; +if (statistic.getMessage() != null) { --- End diff -- Since I added an IF statement the switch statement needs to shift to the right to maintain the code formatting, so that is the reason the format has changed --- 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] carbondata pull request #1142: [CARBONDATA-1271] Enhanced Performance for Hi...
GitHub user bhavya411 opened a pull request: https://github.com/apache/carbondata/pull/1142 [CARBONDATA-1271] Enhanced Performance for Hive Integration with Carbondata There are various fixes added to improve the performance - Added a DictionaryDecodeReadSupport class in Hive Package to directly convert to ArrayWritable which is the default format for Hive - Removed the colids checks as it was hampering performance and causing issues with Map jobs of Hive - Implemented getPos() method to return the rowCount - Made changes in core classes to support Hive Query Statistics - Improved and simplified CarbonHiveSerde - Support for Array Type and Struct Type added in the Integration - Build passed with all test cases executing successfully for Spark 2.1 and Hadoop 2.7.2 - Tested Manually for all the queries You can merge this pull request into a Git repository by running: $ git pull https://github.com/bhavya411/incubator-carbondata CARBONDATA-1271 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/carbondata/pull/1142.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 #1142 commit 9d90aea6af7d433402e809dec957d12563a274c1 Author: Bhavya <bha...@knoldus.com> Date: 2017-07-06T06:23:03Z Fixed performance issue for Hive Integration --- 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] carbondata pull request #984: [CARBONDATA-1008] Make Caron table schema comp...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/984#discussion_r125279885 --- Diff: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java --- @@ -269,9 +270,9 @@ public ICarbonLock getTableUpdateStatusLock() { * @throws Exception */ public String[] getDeleteDeltaFilePath(String blockFilePath) throws Exception { -int tableFactPathLength = CarbonStorePath -.getCarbonTablePath(absoluteTableIdentifier.getStorePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()).getFactDir().length() + 1; +String factTableDir = +absoluteTableIdentifier.getCarbonTableIdentifier().getTableName() + File.separator + "Fact"; +int tableFactPathLength = blockFilePath.indexOf(factTableDir) + factTableDir.length() + 1; --- End diff -- why is this change needed?? --- 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] carbondata pull request #984: [CARBONDATA-1008] Make Caron table schema comp...
Github user bhavya411 commented on a diff in the pull request: https://github.com/apache/carbondata/pull/984#discussion_r125282321 --- Diff: integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala --- @@ -438,11 +457,24 @@ private[sql] case class AlterTableDataTypeChange( schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava) tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0) .setTime_stamp(System.currentTimeMillis) + val sessionState = sparkSession.sessionState.asInstanceOf[CarbonSessionState] AlterTableUtil .updateSchemaInfo(carbonTable, schemaEvolutionEntry, - tableInfo)(sparkSession, - sparkSession.sessionState.asInstanceOf[CarbonSessionState]) + tableInfo)(sparkSession, sessionState) + val useCompatibleSchema = sparkSession.sparkContext.conf + .getBoolean(CarbonCommonConstants.SPARK_SCHEMA_HIVE_COMPATIBILITY_ENABLE, false) + if (useCompatibleSchema) { +val dataTypeInfo = alterTableDataTypeChangeModel.dataTypeInfo +val colSchema = if (dataTypeInfo.dataType == "decimal") { --- End diff -- use constant for "decimal" --- 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] carbondata pull request #875: [CARBONDATA-999] Fixed the Bucketing Issue for...
Github user bhavya411 closed the pull request at: https://github.com/apache/carbondata/pull/875 --- 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] carbondata issue #875: [CARBONDATA-999] Fixed the Bucketing Issue for Spark ...
Github user bhavya411 commented on the issue: https://github.com/apache/carbondata/pull/875 Since the feature is not supported in Spark 1.6 , this PR need not be merged hence closing it. --- 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. ---