diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template index ac2d20edea5..b5f5101ff96 100644 --- a/conf/carbon.properties.template +++ b/conf/carbon.properties.template @@ -23,8 +23,6 @@ carbon.storelocation=hdfs://hacluster/Opt/CarbonStore carbon.ddl.base.hdfs.url=hdfs://hacluster/opt/data #Path where the bad records are stored carbon.badRecords.location=/opt/Carbon/Spark/badrecords -#Mandatory. path to kettle home -carbon.kettle.home=$/carbonlib/carbonplugins #################### Performance Configuration ################## ######## DataLoading Configuration ######## diff --git a/conf/dataload.properties.template b/conf/dataload.properties.template index cfafb4c150a..cab712aae9e 100644 --- a/conf/dataload.properties.template +++ b/conf/dataload.properties.template @@ -20,13 +20,6 @@ # you should change to the code path of your local machine carbon.storelocation=/home/david/Documents/incubator-carbondata/examples/spark2/target/store -#true: use kettle to load data -#false: use new flow to load data -use_kettle=true - -# you should change to the code path of your local machine -carbon.kettle.home=/home/david/Documents/incubator-carbondata/processing/carbonplugins - #csv delimiter character delimiter=, diff --git a/core/pom.xml b/core/pom.xml index 5e46af33727..976f8d27163 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,21 +44,6 @@ carbondata-common ${project.version} - - pentaho-kettle - kettle-engine - ${kettle.version} - - - pentaho-kettle - kettle-core - ${kettle.version} - - - pentaho-kettle - kettle-db - ${kettle.version} - com.google.code.gson gson diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java index d13bc7989c7..789c321e524 100644 --- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java +++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java @@ -68,10 +68,6 @@ public final class CarbonCommonConstants { * default location of the carbon member, hierarchy and fact files */ public static final String STORE_LOCATION_DEFAULT_VAL = "../carbon.store"; - /** - * the folder name of kettle home path - */ - public static final String KETTLE_HOME_NAME = "carbonplugins"; /** * CARDINALITY_INCREMENT_DEFAULT_VALUE */ @@ -1174,9 +1170,6 @@ public final class CarbonCommonConstants { public static final String LOCAL_FILE_PREFIX = "file://"; - public static final String USE_KETTLE = "use_kettle"; - - public static final String USE_KETTLE_DEFAULT = "false"; public static final String CARBON_CUSTOM_BLOCK_DISTRIBUTION = "carbon.custom.block.distribution"; public static final String CARBON_CUSTOM_BLOCK_DISTRIBUTION_DEFAULT = "false"; diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java index e1aa0a0179d..fb21d952c6e 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java @@ -34,7 +34,7 @@ public class CarbonWriteDataHolder { private byte[][] byteValues; /** - * byteValues for no dictionary and non kettle flow. + * byteValues for no dictionary. */ private byte[][][] byteValuesForNonDictionary; @@ -72,7 +72,6 @@ public void reset() { /** * Method to initialise double array - * TODO Remove after kettle flow got removed. * * @param size */ @@ -90,7 +89,7 @@ public void initialiseByteArrayValues(int size) { * * @param size */ - public void initialiseByteArrayValuesWithOutKettle(int size) { + public void initialiseByteArrayValuesForKey(int size) { if (size < 1) { throw new IllegalArgumentException("Invalid array size"); } diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java index 2a5c3421b18..1ddb3e68f8e 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java @@ -345,7 +345,7 @@ protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel blockExecutionInfo .setFixedLengthKeySize(getKeySize(currentBlockQueryDimensions, segmentProperties)); Set dictionaryColumnBlockIndex = new HashSet(); - List noDictionaryColumnBlockIndex = new ArrayList(); + Set noDictionaryColumnBlockIndex = new HashSet(); // get the block index to be read from file for query dimension // for both dictionary columns and no dictionary columns QueryUtil.fillQueryDimensionsBlockIndexes(currentBlockQueryDimensions, diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java index 5a98e44e5bb..be5e8a49e90 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java @@ -607,7 +607,7 @@ public static byte[] getMaskedKey(byte[] data, byte[] maxKey, int[] maskByteRang */ public static void fillQueryDimensionsBlockIndexes(List queryDimensions, Map columnOrdinalToBlockIndexMapping, - Set dictionaryDimensionBlockIndex, List noDictionaryDimensionBlockIndex) { + Set dictionaryDimensionBlockIndex, Set noDictionaryDimensionBlockIndex) { for (QueryDimension queryDimension : queryDimensions) { if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY) && queryDimension.getDimension().numberOfChild() == 0) { diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java index ff82fc62a96..93bf8eb4a9a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java @@ -79,6 +79,15 @@ public byte[] getNoDictionaryKeyByIndex(int index) { return this.noDictionaryKeys[index]; } + /** + * to get the no dictionary column data + * + * @return no dictionary keys + */ + public byte[][] getNoDictionaryKeys() { + return this.noDictionaryKeys; + } + /** * to get the no dictionary column data * diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java index 6d510f62764..bf8c03bc318 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java @@ -23,12 +23,10 @@ import java.io.Closeable; import java.io.DataInputStream; import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStreamReader; import java.io.ObjectInputStream; import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; import java.nio.charset.Charset; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -83,7 +81,6 @@ import org.apache.thrift.protocol.TCompactProtocol; import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.transport.TIOStreamTransport; -import org.pentaho.di.core.exception.KettleException; public final class CarbonUtil { @@ -631,42 +628,6 @@ public static int[] getCardinalityFromLevelMetadataFile(String levelPath) throws return cardinality; } - public static void writeLevelCardinalityFile(String loadFolderLoc, String tableName, - int[] dimCardinality) throws KettleException { - String levelCardinalityFilePath = - loadFolderLoc + File.separator + CarbonCommonConstants.LEVEL_METADATA_FILE + tableName - + CarbonCommonConstants.CARBON_METADATA_EXTENSION; - FileOutputStream fileOutputStream = null; - FileChannel channel = null; - try { - int dimCardinalityArrLength = dimCardinality.length; - - // first four bytes for writing the length of array, remaining for array data - ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE - + dimCardinalityArrLength * CarbonCommonConstants.INT_SIZE_IN_BYTE); - - fileOutputStream = new FileOutputStream(levelCardinalityFilePath); - channel = fileOutputStream.getChannel(); - buffer.putInt(dimCardinalityArrLength); - - for (int i = 0; i < dimCardinalityArrLength; i++) { - buffer.putInt(dimCardinality[i]); - } - - buffer.flip(); - channel.write(buffer); - buffer.clear(); - - LOGGER.info("Level cardinality file written to : " + levelCardinalityFilePath); - } catch (IOException e) { - LOGGER.error("Error while writing level cardinality file : " + levelCardinalityFilePath + e - .getMessage()); - throw new KettleException("Not able to write level cardinality file", e); - } finally { - closeStreams(channel, fileOutputStream); - } - } - /** * From beeline if a delimeter is passed as \001, in code we get it as * escaped string as \\001. So this method will unescape the slash again and diff --git a/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java b/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java deleted file mode 100644 index 1e427aad5e4..00000000000 --- a/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java +++ /dev/null @@ -1,318 +0,0 @@ -/* - * 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.core.writer; - -import java.io.Closeable; -import java.io.File; -import java.io.FileFilter; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonUtil; - -import org.pentaho.di.core.exception.KettleException; - -public class HierarchyValueWriterForCSV { - - /** - * Comment for LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(HierarchyValueWriterForCSV.class.getName()); - /** - * hierarchyName - */ - private String hierarchyName; - - /** - * bufferedOutStream - */ - private FileChannel outPutFileChannel; - - /** - * storeFolderLocation - */ - private String storeFolderLocation; - - /** - * intialized - */ - private boolean intialized; - - /** - * counter the number of files. - */ - private int counter; - - /** - * byteArrayList - */ - private List byteArrayholder = - new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - - /** - * toflush - */ - private int toflush; - - public HierarchyValueWriterForCSV(String hierarchy, String storeFolderLocation) { - this.hierarchyName = hierarchy; - this.storeFolderLocation = storeFolderLocation; - - CarbonProperties instance = CarbonProperties.getInstance(); - - this.toflush = Integer.parseInt(instance - .getProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)); - - int rowSetSize = Integer.parseInt(instance.getProperty(CarbonCommonConstants.GRAPH_ROWSET_SIZE, - CarbonCommonConstants.GRAPH_ROWSET_SIZE_DEFAULT)); - - if (this.toflush > rowSetSize) { - this.toflush = rowSetSize; - } - - updateCounter(hierarchy, storeFolderLocation); - } - - /** - * @return Returns the byteArrayList. - */ - public List getByteArrayList() { - return byteArrayholder; - } - - public FileChannel getBufferedOutStream() { - return outPutFileChannel; - } - - private void updateCounter(final String meString, String storeFolderLocation) { - File storeFolder = new File(storeFolderLocation); - - File[] listFiles = storeFolder.listFiles(new FileFilter() { - - @Override public boolean accept(File file) { - if (file.getName().indexOf(meString) > -1) - - { - return true; - } - return false; - } - }); - - if (null == listFiles || listFiles.length == 0) { - counter = 0; - return; - } - - for (File hierFile : listFiles) { - String hierFileName = hierFile.getName(); - - if (hierFileName.endsWith(CarbonCommonConstants.FILE_INPROGRESS_STATUS)) { - hierFileName = hierFileName.substring(0, hierFileName.lastIndexOf('.')); - try { - counter = Integer.parseInt(hierFileName.substring(hierFileName.length() - 1)); - } catch (NumberFormatException nfe) { - - if (new File(hierFileName + '0' + CarbonCommonConstants.LEVEL_FILE_EXTENSION).exists()) { - // Need to skip because the case can come in which server went down while files were - // merging and the other hierarchy files were not deleted, and the current file - // status is inrogress. so again we will merge the files and rename to normal file - LOGGER.info("Need to skip as this can be case in which hierarchy file already renamed"); - if (hierFile.delete()) { - LOGGER.info("Deleted the Inprogress hierarchy Files."); - } - } else { - // levelfileName0.level file not exist that means files is merged and other - // files got deleted. while renaming this file from inprogress to normal file, - // server got restarted/killed. so we need to rename the file to normal. - - File inprogressFile = new File(storeFolder + File.separator + hierFile.getName()); - File changetoName = new File(storeFolder + File.separator + hierFileName); - - if (inprogressFile.renameTo(changetoName)) { - LOGGER.info( - "Renaming the level Files while creating the new instance on server startup."); - } - - } - - } - } - - String val = hierFileName.substring(hierFileName.length() - 1); - - int parsedVal = getIntValue(val); - - if (counter < parsedVal) { - counter = parsedVal; - } - } - counter++; - } - - private int getIntValue(String val) { - int parsedVal = 0; - try { - parsedVal = Integer.parseInt(val); - } catch (NumberFormatException nfe) { - LOGGER.info("Hierarchy File is already renamed so there will not be" - + "any need to keep the counter"); - } - return parsedVal; - } - - private void intialize() throws KettleException { - intialized = true; - - File f = new File(storeFolderLocation + File.separator + hierarchyName + counter - + CarbonCommonConstants.FILE_INPROGRESS_STATUS); - - counter++; - - FileOutputStream fos = null; - - boolean isFileCreated = false; - if (!f.exists()) { - try { - isFileCreated = f.createNewFile(); - - } catch (IOException e) { - //not required: findbugs fix - throw new KettleException("unable to create member mapping file", e); - } - if (!isFileCreated) { - throw new KettleException("unable to create file" + f.getAbsolutePath()); - } - } - - try { - fos = new FileOutputStream(f); - - outPutFileChannel = fos.getChannel(); - } catch (FileNotFoundException e) { - closeStreamAndDeleteFile(f, outPutFileChannel, fos); - throw new KettleException("member Mapping File not found to write mapping info", e); - } - } - - public void writeIntoHierarchyFile(byte[] bytes, int primaryKey) throws KettleException { - if (!intialized) { - intialize(); - } - - ByteBuffer byteBuffer = storeValueInCache(bytes, primaryKey); - - try { - byteBuffer.flip(); - outPutFileChannel.write(byteBuffer); - } catch (IOException e) { - throw new KettleException("Error while writting in the hierarchy mapping file", e); - } - } - - private ByteBuffer storeValueInCache(byte[] bytes, int primaryKey) { - - // adding 4 to store the total length of the row at the beginning - ByteBuffer buffer = ByteBuffer.allocate(bytes.length + 4); - - buffer.put(bytes); - buffer.putInt(primaryKey); - - return buffer; - } - - public void performRequiredOperation() throws KettleException { - if (byteArrayholder.size() == 0) { - return; - } - //write to the file and close the stream. - Collections.sort(byteArrayholder); - - for (ByteArrayHolder byteArray : byteArrayholder) { - writeIntoHierarchyFile(byteArray.getMdKey(), byteArray.getPrimaryKey()); - } - - CarbonUtil.closeStreams(outPutFileChannel); - - //rename the inprogress file to normal .level file - String filePath = this.storeFolderLocation + File.separator + hierarchyName + (counter - 1) - + CarbonCommonConstants.FILE_INPROGRESS_STATUS; - File inProgressFile = new File(filePath); - String inprogressFileName = inProgressFile.getName(); - - String changedFileName = inprogressFileName.substring(0, inprogressFileName.lastIndexOf('.')); - - File orgFinalName = new File(this.storeFolderLocation + File.separator + changedFileName); - - if (!inProgressFile.renameTo(orgFinalName)) { - LOGGER.error("Not able to rename file : " + inprogressFileName); - } - - //create the new outputStream - try { - intialize(); - } catch (KettleException e) { - LOGGER.error("Not able to create output stream for file:" + hierarchyName + (counter - 1)); - } - - //clear the byte array holder also. - byteArrayholder.clear(); - } - - private void closeStreamAndDeleteFile(File f, Closeable... streams) { - boolean isDeleted = false; - for (Closeable stream : streams) { - if (null != stream) { - try { - stream.close(); - } catch (IOException e) { - LOGGER.error(e, "unable to close the stream "); - } - - } - } - - // delete the file - isDeleted = f.delete(); - if (!isDeleted) { - LOGGER.error("Unable to delete the file " + f.getAbsolutePath()); - } - - } - - public String getHierarchyName() { - return hierarchyName; - } - - public int getCounter() { - return counter; - } - -} - diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java index fed712eb095..5c51c87f0b2 100644 --- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java +++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java @@ -35,9 +35,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import static org.pentaho.di.core.util.Assert.assertNull; public class DFSFileHolderImplUnitTest { diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java index da722cf196e..ed50d63ba88 100644 --- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java +++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java @@ -35,9 +35,8 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertThat; -import static org.pentaho.di.core.util.Assert.assertNull; public class FileHolderImplUnitTest { diff --git a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java index 9f37f784a9b..e882f4eb64d 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java @@ -21,8 +21,8 @@ import org.junit.BeforeClass; import org.junit.Test; -import static org.pentaho.di.core.util.Assert.assertNotNull; -import static org.pentaho.di.core.util.Assert.assertNull; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; public class ArrayQueryTypeTest { diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java index 30c8236c8db..2cdb46ec4df 100644 --- a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java +++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java @@ -35,8 +35,8 @@ import static org.hamcrest.core.Is.is; import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; -import static org.pentaho.di.core.util.Assert.assertFalse; -import static org.pentaho.di.core.util.Assert.assertNull; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; public class ExpressionResultTest { private static final double DELTA = 1e-15; diff --git a/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java b/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java index 90cae835535..bb892a90668 100644 --- a/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java +++ b/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java @@ -23,7 +23,7 @@ import org.junit.Test; import static junit.framework.TestCase.assertNotNull; -import static org.pentaho.di.core.util.Assert.assertNull; +import static org.junit.Assert.assertNull; public class DriverQueryStatisticsRecorderImplTest { private static DriverQueryStatisticsRecorderImpl driverQueryStatisticsRecorderImpl = null; diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java index 9beaac7e8e9..9adf4d407ed 100644 --- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java +++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java @@ -16,12 +16,26 @@ */ package org.apache.carbondata.core.util; -import mockit.Mock; -import mockit.MockUp; +import java.io.BufferedReader; +import java.io.DataInputStream; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; -import org.apache.carbondata.core.metadata.ColumnarFormatVersion; import org.apache.carbondata.core.datastore.block.TableBlockInfo; import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk; +import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel; +import org.apache.carbondata.core.datastore.compression.WriterCompressModel; +import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile; +import org.apache.carbondata.core.datastore.impl.FileFactory; +import org.apache.carbondata.core.metadata.ColumnarFormatVersion; +import org.apache.carbondata.core.metadata.ValueEncoderMeta; import org.apache.carbondata.core.metadata.blocklet.DataFileFooter; import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk; import org.apache.carbondata.core.metadata.datatype.DataType; @@ -29,29 +43,20 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel; -import org.apache.carbondata.core.datastore.compression.WriterCompressModel; -import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.metadata.ValueEncoderMeta; import org.apache.carbondata.core.scan.model.QueryDimension; +import mockit.Mock; +import mockit.MockUp; import org.apache.hadoop.security.UserGroupInformation; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; -import org.pentaho.di.core.exception.KettleException; - -import java.io.*; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import static org.hamcrest.MatcherAssert.assertThat; -import static junit.framework.TestCase.*; +import static junit.framework.TestCase.assertEquals; +import static junit.framework.TestCase.assertTrue; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; public class CarbonUtilTest { @@ -300,23 +305,6 @@ public void testToDeleteFoldersAndFilesSintlyWithInterruptedException() assertEquals(result, 5); } - @Test public void testToWriteLevelCardinalityFile() throws KettleException { - int[] dimCardinality = { 10, 20, 30, 40 }; - CarbonUtil.writeLevelCardinalityFile("../core/src/test/resources/testDatabase", "testTable", - dimCardinality); - assertTrue(new File("../core/src/test/resources/testDatabase/levelmetadata_testTable.metadata") - .exists()); - } - - @Test public void testToGetCardinalityFromLevelMetadataFile() - throws IOException, InterruptedException { - int[] cardinality = CarbonUtil.getCardinalityFromLevelMetadataFile( - "../core/src/test/resources/testDatabase/levelmetadata_testTable.metadata"); - int[] expectedCardinality = { 10, 20, 30, 40 }; - for (int i = 0; i < cardinality.length; i++) { - assertEquals(cardinality[i], expectedCardinality[i]); - } - } @Test public void testToGetCardinalityFromLevelMetadataFileForInvalidPath() throws IOException, InterruptedException { diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala index b1d9e411c7b..36013a8724e 100644 --- a/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala +++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala @@ -41,9 +41,6 @@ object CarbonExample { STORED BY 'carbondata' """) - // Currently there are two data loading flows in CarbonData, one uses Kettle as ETL tool - // in each node to do data loading, another uses a multi-thread framework without Kettle (See - // AbstractDataLoadProcessorStep) // Load data cc.sql(s""" LOAD DATA LOCAL INPATH '$testData' into table t3 diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala index 41912bcc8f4..f98ec3bc673 100644 --- a/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala +++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala @@ -31,7 +31,6 @@ object ExampleUtils { def currentPath: String = new File(this.getClass.getResource("/").getPath + "../../") .getCanonicalPath val storeLocation = currentPath + "/target/store" - val kettleHome = new File(currentPath + "/../../processing/carbonplugins").getCanonicalPath def createCarbonContext(appName: String): CarbonContext = { val sc = new SparkContext(new SparkConf() @@ -44,12 +43,7 @@ object ExampleUtils { val cc = new CarbonContext(sc, storeLocation, currentPath + "/target/carbonmetastore") CarbonProperties.getInstance() - .addProperty("carbon.kettle.home", kettleHome) .addProperty("carbon.storelocation", storeLocation) - // whether use table split partition - // true -> use table split partition, support multiple partition loading - // false -> use node split partition, support data load by host partition - CarbonProperties.getInstance().addProperty("carbon.table.split.partition.enable", "false") cc } @@ -90,7 +84,6 @@ object ExampleUtils { .format("carbondata") .option("tableName", tableName) .option("compress", "true") - .option("use_kettle", "false") .option("tempCSV", "false") .mode(mode) .save() diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala index 1c3b7f066e3..2ae93414a60 100644 --- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala +++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala @@ -34,7 +34,6 @@ object CarbonSessionExample { val metastoredb = s"$rootPath/examples/spark2/target" CarbonProperties.getInstance() - .addProperty("carbon.kettle.home", s"$rootPath/processing/carbonplugins") .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd") import org.apache.spark.sql.CarbonSession._ diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala index 381b2e89f80..63e680a2857 100644 --- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala +++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala @@ -54,7 +54,6 @@ object SparkSessionExample { .getOrCreate() CarbonProperties.getInstance() - .addProperty("carbon.kettle.home", s"$rootPath/processing/carbonplugins") .addProperty("carbon.storelocation", storeLocation) spark.sparkContext.setLogLevel("WARN") diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java index b55875f986a..0d751fc3609 100644 --- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java +++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java @@ -28,11 +28,11 @@ import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.hadoop.CarbonInputFormat; import org.apache.carbondata.hadoop.CarbonProjection; -import org.apache.carbondata.hadoop.test.util.StoreCreator; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; +import org.apache.carbondata.hadoop.test.util.StoreCreator; import junit.framework.TestCase; import org.apache.hadoop.conf.Configuration; @@ -87,6 +87,7 @@ public class CarbonInputMapperTest extends TestCase { Assert.assertEquals("Count lines are not matching", 1000, countTheLines(outPath)); Assert.assertEquals("Column count are not matching", 3, countTheColumns(outPath)); } catch (Exception e) { + e.printStackTrace(); Assert.assertTrue("failed", false); } } diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java index 532cd432f38..51ce2c582d8 100644 --- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java +++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java @@ -16,19 +16,37 @@ */ package org.apache.carbondata.hadoop.test.util; -import com.google.gson.Gson; -import org.apache.hadoop.fs.Path; - +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.Charset; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import org.apache.carbondata.common.CarbonIterator; 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.CarbonCommonConstants; +import org.apache.carbondata.core.datastore.impl.FileFactory; +import org.apache.carbondata.core.fileoperations.AtomicFileOperations; +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl; +import org.apache.carbondata.core.fileoperations.FileWriteOperation; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; +import org.apache.carbondata.core.metadata.CarbonMetadata; import org.apache.carbondata.core.metadata.CarbonTableIdentifier; import org.apache.carbondata.core.metadata.ColumnIdentifier; -import org.apache.carbondata.core.metadata.CarbonMetadata; import org.apache.carbondata.core.metadata.converter.SchemaConverter; import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl; import org.apache.carbondata.core.metadata.datatype.DataType; @@ -42,14 +60,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.util.path.CarbonStorePath; -import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.processing.csvreaderstep.BlockDetails; import org.apache.carbondata.core.statusmanager.LoadMetadataDetails; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.path.CarbonStorePath; +import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.core.writer.CarbonDictionaryWriter; import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl; import org.apache.carbondata.core.writer.ThriftWriter; @@ -57,25 +72,29 @@ import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl; import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo; import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator; -import org.apache.carbondata.core.fileoperations.AtomicFileOperations; -import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl; -import org.apache.carbondata.core.fileoperations.FileWriteOperation; -import org.apache.carbondata.processing.api.dataloader.DataLoadModel; import org.apache.carbondata.processing.api.dataloader.SchemaInfo; import org.apache.carbondata.processing.constants.TableOptionConstant; -import org.apache.carbondata.processing.csvload.DataGraphExecuter; -import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus; -import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus; -import org.apache.carbondata.processing.graphgenerator.GraphGenerator; -import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException; +import org.apache.carbondata.processing.csvload.BlockDetails; +import org.apache.carbondata.processing.csvload.CSVInputFormat; +import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator; +import org.apache.carbondata.processing.csvload.StringArrayWritable; +import org.apache.carbondata.processing.model.CarbonDataLoadSchema; +import org.apache.carbondata.processing.model.CarbonLoadModel; +import org.apache.carbondata.processing.newflow.DataLoadExecutor; +import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants; -import java.io.*; -import java.nio.charset.Charset; -import java.text.SimpleDateFormat; -import java.util.*; +import com.google.gson.Gson; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.TaskAttemptID; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; /** * This class will create store file based on provided schema + * */ public class StoreCreator { @@ -104,26 +123,52 @@ public static void createCarbonStore() { try { - String factFilePath = new File("src/test/resources/data.csv").getCanonicalPath(); + String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath(); File storeDir = new File(absoluteTableIdentifier.getStorePath()); CarbonUtil.deleteFoldersAndFiles(storeDir); CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, absoluteTableIdentifier.getStorePath()); - String kettleHomePath = "../processing/carbonplugins"; CarbonTable table = createTable(); writeDictionary(factFilePath, table); CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table); - LoadModel loadModel = new LoadModel(); + CarbonLoadModel loadModel = new CarbonLoadModel(); String partitionId = "0"; - loadModel.setSchema(schema); + loadModel.setCarbonDataLoadSchema(schema); loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName()); loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName()); loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName()); loadModel.setFactFilePath(factFilePath); loadModel.setLoadMetadataDetails(new ArrayList()); + loadModel.setStorePath(absoluteTableIdentifier.getStorePath()); + loadModel.setDateFormat(null); + loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty( + CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, + CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)); + loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty( + CarbonCommonConstants.CARBON_DATE_FORMAT, + CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)); + loadModel + .setSerializationNullFormat( + TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N"); + loadModel + .setBadRecordsLoggerEnable( + TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false"); + loadModel + .setBadRecordsAction( + TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE"); + loadModel + .setIsEmptyDataBadRecord( + DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false"); + loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary"); + loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(",")); + loadModel.setTaskNo("0"); + loadModel.setSegmentId("0"); + loadModel.setPartitionId("0"); + loadModel.setFactTimeStamp(System.currentTimeMillis()); + + executeGraph(loadModel, absoluteTableIdentifier.getStorePath()); - executeGraph(loadModel, absoluteTableIdentifier.getStorePath(), kettleHomePath); } catch (Exception e) { e.printStackTrace(); } @@ -311,12 +356,10 @@ private static void writeDictionary(String factFilePath, CarbonTable table) thro * * @param loadModel * @param storeLocation - * @param kettleHomePath * @throws Exception */ - public static void executeGraph(LoadModel loadModel, String storeLocation, String kettleHomePath) + public static void executeGraph(CarbonLoadModel loadModel, String storeLocation) throws Exception { - System.setProperty("KETTLE_HOME", kettleHomePath); new File(storeLocation).mkdirs(); String outPutLoc = storeLocation + "/etl"; String databaseName = loadModel.getDatabaseName(); @@ -344,35 +387,39 @@ public static void executeGraph(LoadModel loadModel, String storeLocation, Strin path.delete(); } - DataProcessTaskStatus dataProcessTaskStatus = new DataProcessTaskStatus(databaseName, tableName); - dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath()); SchemaInfo info = new SchemaInfo(); BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()), 0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"}); - GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails }); - dataProcessTaskStatus.setBlocksID("qwqwq"); - dataProcessTaskStatus.setEscapeCharacter("\\"); - dataProcessTaskStatus.setQuoteCharacter("\""); - dataProcessTaskStatus.setCommentCharacter("#"); - dataProcessTaskStatus.setDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT); + Configuration configuration = new Configuration(); + CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar()); + CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter()); + CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar()); + CSVInputFormat.setHeaderExtractionEnabled(configuration, true); + CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar()); + CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance() + .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE, + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT)); + + TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0)); + CSVInputFormat format = new CSVInputFormat(); + + RecordReader recordReader = + format.createRecordReader(blockDetails, hadoopAttemptContext); + + CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext); + new DataLoadExecutor().execute(loadModel, + storeLocation, + new CarbonIterator[]{readerIterator}); + info.setDatabaseName(databaseName); info.setTableName(tableName); - info.setSerializationNullFormat( - TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N"); - info.setBadRecordsLoggerEnable( - TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false"); - info.setBadRecordsLoggerAction( - TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "force"); - - generateGraph(dataProcessTaskStatus, info, loadModel.getTableName(), "0", loadModel.getSchema(), null, - loadModel.getLoadMetadataDetails()); - - DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus); - graphExecuter - .executeGraph(graphPath, info, loadModel.getSchema()); + +// DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus); +// graphExecuter +// .executeGraph(graphPath, info, loadModel.getSchema()); // LoadMetadataDetails[] loadDetails = // CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath()); - writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(), + writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(), new ArrayList()); String segLocation = @@ -403,6 +450,7 @@ public static void executeGraph(LoadModel loadModel, String storeLocation, Strin public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName, String tableName, List listOfLoadFolderDetails) throws IOException { LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails(); + loadMetadataDetails.setLoadEndTime(System.currentTimeMillis()); loadMetadataDetails.setLoadStatus("SUCCESS"); loadMetadataDetails.setLoadName(String.valueOf(0)); loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime())); @@ -442,40 +490,6 @@ public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databas } - /** - * generate graph - * - * @param dataProcessTaskStatus - * @param info - * @param tableName - * @param partitionID - * @param schema - * @param factStoreLocation - * @param loadMetadataDetails - * @throws GraphGeneratorException - */ - private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info, - String tableName, String partitionID, CarbonDataLoadSchema schema, String factStoreLocation, - List loadMetadataDetails) - throws GraphGeneratorException { - DataLoadModel model = new DataLoadModel(); - model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath() || null != dataProcessTaskStatus.getFilesToProcess()); - model.setSchemaInfo(info); - model.setTableName(dataProcessTaskStatus.getTableName()); - model.setTaskNo("1"); - model.setBlocksID(dataProcessTaskStatus.getBlocksID()); - model.setFactTimeStamp(System.currentTimeMillis() + ""); - model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter()); - model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter()); - model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter()); - model.setDateFormat(dataProcessTaskStatus.getDateFormat()); - String outputLocation = CarbonProperties.getInstance() - .getProperty("store_output_location", "../carbon-store/system/carbon/etl"); - GraphGenerator generator = - new GraphGenerator(model, partitionID, factStoreLocation, schema, "0", outputLocation); - generator.generateGraph(); - } - public static String readCurrentTime() { SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP); String date = null; @@ -485,60 +499,6 @@ public static String readCurrentTime() { return date; } - /** - * This is local model object used inside this class to store information related to data loading - */ - private static class LoadModel { - - private CarbonDataLoadSchema schema; - private String tableName; - private String databaseName; - private List loadMetaDetail; - private String factFilePath; - - public void setSchema(CarbonDataLoadSchema schema) { - this.schema = schema; - } - - public List getLoadMetadataDetails() { - return loadMetaDetail; - } - - public CarbonDataLoadSchema getSchema() { - return schema; - } - - public String getFactFilePath() { - return factFilePath; - } - - public String getTableName() { - return tableName; - } - - public String getDatabaseName() { - return databaseName; - } - - public void setLoadMetadataDetails(List loadMetaDetail) { - this.loadMetaDetail = loadMetaDetail; - } - - public void setFactFilePath(String factFilePath) { - this.factFilePath = factFilePath; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - - } - public static void main(String[] args) { StoreCreator.createCarbonStore(); } diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml index d92bc943fd7..f9bfaa05d1f 100644 --- a/integration/spark-common-test/pom.xml +++ b/integration/spark-common-test/pom.xml @@ -147,7 +147,6 @@ true - ${use.kettle} diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala index aa18b8f1b79..58b49d45afe 100644 --- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala +++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala @@ -136,7 +136,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll { STORED BY 'org.apache.carbondata.format' """) val testData = s"$resourcesPath/sample.csv" try { - sql(s"LOAD DATA LOCAL INPATH '$testData' into table load_test_singlepass options ('USE_KETTLE'='FALSE','SINGLE_PASS'='TRUE')") + sql(s"LOAD DATA LOCAL INPATH '$testData' into table load_test_singlepass options ('SINGLE_PASS'='TRUE')") } catch { case ex: Exception => assert(false) diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala index 052ee455512..7df7fcdd7c4 100644 --- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala +++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala @@ -25,7 +25,7 @@ class DoubleDataTypeTest extends QueryTest with BeforeAndAfterAll { override def beforeAll: Unit = { sql("DROP TABLE IF EXISTS oscon_carbon_old") sql("""create table oscon_carbon_old (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CUST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_CNT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RET_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' tblproperties('DICTIONARY_EXCLUDE'='CUST_ID,CUST_NICK_NAME,CUST_FIRST_NAME,CUST_LAST_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_MDL_ID,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,DEVICE_NAME,PROD_BAR_CODE,ITM_ID,ITM_NAME,ITM_BRAND_ID,ITM_BRAND,BOM,PACKING_LIST_NO,TRACKING_NO,ACTIVE_FIRMWARE_VER,LATEST_FIRMWARE_VER,LATEST_EMUI_VERSION,LATEST_NETWORK,STR_ORDER_NO','table_blocksize'='256')""") - sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old options('USE_KETTLE'='true','DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, ITM_CATEGORY_ID, ITM_CLASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_LIVE, PROD_LOC, PROD_LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRICE, STR_SALES_PRICE, STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPON_AMT, OL_EXT_SHIP_CST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""") + sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old options('DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, ITM_CATEGORY_ID, ITM_CLASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_LIVE, PROD_LOC, PROD_LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRICE, STR_SALES_PRICE, STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPON_AMT, OL_EXT_SHIP_CST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""") } test("test to check result for double data type") { diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala index 8670d8f50d7..353db9e4781 100644 --- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala +++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala @@ -93,8 +93,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll { "row format delimited fields terminated by ','" ) - sql(s"LOAD DATA local inpath '$resourcesPath/data.csv' INTO table smallinttable " + - "OPTIONS('USE_KETTLE'='false')") + sql(s"LOAD DATA local inpath '$resourcesPath/data.csv' INTO table smallinttable ") sql(s"LOAD DATA local inpath '$resourcesPath/datawithoutheader.csv' overwrite " + "INTO table smallinthivetable") diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java index 85e3cd73d94..cc16398b617 100644 --- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java +++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java @@ -67,13 +67,6 @@ import org.apache.carbondata.core.util.CarbonUtil; import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.carbondata.processing.api.dataloader.DataLoadModel; -import org.apache.carbondata.processing.api.dataloader.SchemaInfo; -import org.apache.carbondata.processing.csvload.DataGraphExecuter; -import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus; -import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus; -import org.apache.carbondata.processing.graphgenerator.GraphGenerator; -import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException; import org.apache.carbondata.processing.model.CarbonLoadModel; import org.apache.carbondata.spark.merger.NodeBlockRelation; import org.apache.carbondata.spark.merger.NodeMultiBlockRelation; @@ -90,97 +83,7 @@ public final class CarbonLoaderUtil { private CarbonLoaderUtil() { } - private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info, - CarbonLoadModel loadModel, String outputLocation) - throws GraphGeneratorException { - DataLoadModel model = new DataLoadModel(); - model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath() - || null != dataProcessTaskStatus.getFilesToProcess()); - model.setSchemaInfo(info); - model.setTableName(dataProcessTaskStatus.getTableName()); - List loadMetadataDetails = loadModel.getLoadMetadataDetails(); - model.setBlocksID(dataProcessTaskStatus.getBlocksID()); - model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter()); - model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter()); - model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter()); - model.setRddIteratorKey(dataProcessTaskStatus.getRddIteratorKey()); - model.setTaskNo(loadModel.getTaskNo()); - model.setFactTimeStamp(loadModel.getFactTimeStamp()); - model.setMaxColumns(loadModel.getMaxColumns()); - model.setDateFormat(loadModel.getDateFormat()); - GraphGenerator generator = new GraphGenerator(model, loadModel.getPartitionId(), - loadModel.getStorePath(), loadModel.getCarbonDataLoadSchema(), loadModel.getSegmentId(), - outputLocation); - generator.generateGraph(); - } - public static void executeGraph(CarbonLoadModel loadModel, String storeLocation, - String storePath, String kettleHomePath) throws Exception { - System.setProperty("KETTLE_HOME", kettleHomePath); - if (!new File(storeLocation).mkdirs()) { - LOGGER.error("Error while creating the temp store path: " + storeLocation); - } - String outPutLoc = storeLocation + "/etl"; - String databaseName = loadModel.getDatabaseName(); - String tableName = loadModel.getTableName(); - String tempLocationKey = databaseName + CarbonCommonConstants.UNDERSCORE + tableName - + CarbonCommonConstants.UNDERSCORE + loadModel.getTaskNo(); - CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation); - CarbonProperties.getInstance() - .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, storePath); - // CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc); - CarbonProperties.getInstance().addProperty("send.signal.load", "false"); - - String fileNamePrefix = ""; - if (loadModel.isAggLoadRequest()) { - fileNamePrefix = "graphgenerator"; - } - String graphPath = - outPutLoc + File.separator + databaseName + File.separator + tableName + File.separator - + loadModel.getSegmentId() + File.separator + loadModel.getTaskNo() + File.separator - + tableName + fileNamePrefix + ".ktr"; - File path = new File(graphPath); - if (path.exists()) { - path.delete(); - } - - DataProcessTaskStatus dataProcessTaskStatus - = new DataProcessTaskStatus(databaseName, tableName); - dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath()); - if (loadModel.isDirectLoad()) { - dataProcessTaskStatus.setFilesToProcess(loadModel.getFactFilesToProcess()); - dataProcessTaskStatus.setDirectLoad(true); - dataProcessTaskStatus.setCsvDelimiter(loadModel.getCsvDelimiter()); - dataProcessTaskStatus.setCsvHeader(loadModel.getCsvHeader()); - } - - dataProcessTaskStatus.setBlocksID(loadModel.getBlocksID()); - dataProcessTaskStatus.setEscapeCharacter(loadModel.getEscapeChar()); - dataProcessTaskStatus.setQuoteCharacter(loadModel.getQuoteChar()); - dataProcessTaskStatus.setCommentCharacter(loadModel.getCommentChar()); - dataProcessTaskStatus.setRddIteratorKey(loadModel.getRddIteratorKey()); - dataProcessTaskStatus.setDateFormat(loadModel.getDateFormat()); - SchemaInfo info = new SchemaInfo(); - info.setDatabaseName(databaseName); - info.setTableName(tableName); - info.setAutoAggregateRequest(loadModel.isAggLoadRequest()); - info.setComplexDelimiterLevel1(loadModel.getComplexDelimiterLevel1()); - info.setComplexDelimiterLevel2(loadModel.getComplexDelimiterLevel2()); - info.setSerializationNullFormat(loadModel.getSerializationNullFormat()); - info.setBadRecordsLoggerEnable(loadModel.getBadRecordsLoggerEnable()); - info.setBadRecordsLoggerAction(loadModel.getBadRecordsAction()); - - generateGraph(dataProcessTaskStatus, info, loadModel, outPutLoc); - - DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus); - graphExecuter - .executeGraph(graphPath, info, loadModel.getCarbonDataLoadSchema()); - } - - public static List addNewSliceNameToList(String newSlice, List activeSlices) { - activeSlices.add(newSlice); - return activeSlices; - } public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) { CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable(); @@ -252,43 +155,6 @@ private static void deleteStorePath(String path) { } } - public static List getListOfValidSlices(LoadMetadataDetails[] details) { - List activeSlices = - new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - for (LoadMetadataDetails oneLoad : details) { - if (CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS.equals(oneLoad.getLoadStatus()) - || CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS.equals(oneLoad.getLoadStatus()) - || CarbonCommonConstants.MARKED_FOR_UPDATE.equals(oneLoad.getLoadStatus())) { - if (null != oneLoad.getMergedLoadName()) { - String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getMergedLoadName(); - activeSlices.add(loadName); - } else { - String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getLoadName(); - activeSlices.add(loadName); - } - } - } - return activeSlices; - } - - public static List getListOfUpdatedSlices(LoadMetadataDetails[] details) { - List updatedSlices = - new ArrayList(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - for (LoadMetadataDetails oneLoad : details) { - if (CarbonCommonConstants.MARKED_FOR_UPDATE.equals(oneLoad.getLoadStatus())) { - if (null != oneLoad.getMergedLoadName()) { - updatedSlices.add(oneLoad.getMergedLoadName()); - } else { - updatedSlices.add(oneLoad.getLoadName()); - } - } - } - return updatedSlices; - } - - public static void removeSliceFromMemory(String databaseName, String tableName, String loadName) { - // TODO: Remove from memory - } /** * This method will delete the local data load folder location after data load is complete diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java index dd41c59e886..29aa7e75fcb 100644 --- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java +++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java @@ -46,13 +46,13 @@ import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.merger.exeception.SliceMergerException; import org.apache.carbondata.processing.model.CarbonLoadModel; import org.apache.carbondata.processing.store.CarbonDataFileAttributes; import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar; import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel; import org.apache.carbondata.processing.store.CarbonFactHandler; import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException; +import org.apache.carbondata.spark.merger.exeception.SliceMergerException; /** * This is the Merger class responsible for the merging of the segments. diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java index c65dab6a00e..08b563ff3f8 100644 --- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java +++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java @@ -16,12 +16,8 @@ */ package org.apache.carbondata.spark.merger; -import java.util.ArrayList; -import java.util.List; - import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper; -import org.apache.carbondata.processing.util.NonDictionaryUtil; /** * This class will be used to convert the Result into the format used in data writer. @@ -61,15 +57,7 @@ public Object[] getObjectArray(Object[] carbonTuple) { // put No dictionary byte [] if (isNoDictionaryPresent) { - - int noDicCount = segmentproperties.getNumberOfNoDictionaryDimension(); - List noDicByteArr = new ArrayList<>(noDicCount); - for (int i = 0; i < noDicCount; i++) { - noDicByteArr.add(((ByteArrayWrapper) carbonTuple[0]).getNoDictionaryKeyByIndex(i)); - } - byte[] singleByteArr = NonDictionaryUtil.convertListByteArrToSingleArr(noDicByteArr); - - row[index++] = singleByteArr; + row[index++] = ((ByteArrayWrapper) carbonTuple[0]).getNoDictionaryKeys(); } // put No Dictionary Dims diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java similarity index 96% rename from processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java rename to integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java index 3ae360459aa..fd6610c70b8 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java +++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.processing.merger.exeception; +package org.apache.carbondata.spark.merger.exeception; import java.util.Locale; diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java index ae8f7842cf5..297bd7d042e 100644 --- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java +++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java @@ -16,12 +16,7 @@ */ /** - * Project Name : Carbon - * Module Name : CARBON Data Processor - * Created Date : 15-Sep-2015 - * FileName : LoadMetadataUtil.java - * Description : Kettle step to generate MD Key - * Class Version : 1.0 + * Utility for load data */ package org.apache.carbondata.spark.util; diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala index f8f11d6f023..c29c1a2518d 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala @@ -44,8 +44,6 @@ class CarbonOption(options: Map[String, String]) { def compress: Boolean = options.getOrElse("compress", "false").toBoolean - def useKettle: Boolean = options.getOrElse("use_kettle", "false").toBoolean - def singlePass: Boolean = options.getOrElse("single_pass", "false").toBoolean def dictionaryInclude: Option[String] = options.get("dictionary_include") diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala deleted file mode 100644 index e3f2cf629e8..00000000000 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala +++ /dev/null @@ -1,754 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.carbondata.spark.rdd - -import java.lang.Long -import java.nio.ByteBuffer -import java.text.SimpleDateFormat -import java.util -import java.util.UUID - -import scala.collection.JavaConverters._ -import scala.util.Random - -import org.apache.spark.{Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionWrap, RDD} -import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.command.ExecutionErrors -import org.apache.spark.util.SparkUtil - -import org.apache.carbondata.common.CarbonIterator -import org.apache.carbondata.common.logging.LogServiceFactory -import org.apache.carbondata.common.logging.impl.StandardLogService -import org.apache.carbondata.core.constants.CarbonCommonConstants -import org.apache.carbondata.core.statusmanager.LoadMetadataDetails -import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory} -import org.apache.carbondata.processing.constants.DataProcessorConstants -import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInputUtils} -import org.apache.carbondata.processing.etl.DataLoadingException -import org.apache.carbondata.processing.graphgenerator.GraphGenerator -import org.apache.carbondata.processing.model.CarbonLoadModel -import org.apache.carbondata.spark.DataLoadResult -import org.apache.carbondata.spark.load._ -import org.apache.carbondata.spark.splits.TableSplit -import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil} - -/** - * This partition class use to split by TableSplit - * - */ -class CarbonTableSplitPartition(rddId: Int, val idx: Int, @transient val tableSplit: TableSplit, - val blocksDetails: Array[BlockDetails]) - extends Partition { - - override val index: Int = idx - val serializableHadoopSplit = new SerializableWritable[TableSplit](tableSplit) - val partitionBlocksDetail = blocksDetails - - override def hashCode(): Int = 41 * (41 + rddId) + idx -} - -/** - * This partition class use to split by Host - * - */ -class CarbonNodePartition(rddId: Int, val idx: Int, host: String, - val blocksDetails: Array[BlockDetails]) - extends Partition { - - override val index: Int = idx - val serializableHadoopSplit = host - val nodeBlocksDetail = blocksDetails - - override def hashCode(): Int = 41 * (41 + rddId) + idx -} - -class SparkPartitionLoader(model: CarbonLoadModel, - splitIndex: Int, - storePath: String, - kettleHomePath: String, - loadCount: String, - loadMetadataDetails: LoadMetadataDetails) { - private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName) - - var storeLocation: String = "" - - def initialize(): Unit = { - val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null) - if (null == carbonPropertiesFilePath) { - System.setProperty("carbon.properties.filepath", - System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties") - } - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId) - CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true") - CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1") - CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true") - CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true") - CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true") - CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000") - CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false") - CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000") - - // this property is used to determine whether temp location for carbon is inside - // container temp dir or is yarn application directory. - val carbonUseLocalDir = CarbonProperties.getInstance() - .getProperty("carbon.use.local.dir", "false") - if (carbonUseLocalDir.equalsIgnoreCase("true")) { - val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf) - if (null != storeLocations && storeLocations.nonEmpty) { - storeLocation = storeLocations(Random.nextInt(storeLocations.length)) - } - if (storeLocation == null) { - storeLocation = System.getProperty("java.io.tmpdir") - } - } else { - storeLocation = System.getProperty("java.io.tmpdir") - } - storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex - } - - def run(): Unit = { - try { - CarbonLoaderUtil.executeGraph(model, storeLocation, storePath, - kettleHomePath) - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS) - } catch { - case e: DataLoadingException => if (e.getErrorCode == - DataProcessorConstants.BAD_REC_FOUND) { - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS) - LOGGER.info("Bad Record Found") - } else { - throw e - } - case e: Exception => - throw e - } finally { - // delete temp location data - try { - val isCompaction = false - CarbonLoaderUtil.deleteLocalDataLoadFolderLocation(model, isCompaction) - } catch { - case e: Exception => - LOGGER.error(e, "Failed to delete local data") - } - if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals( - loadMetadataDetails.getLoadStatus)) { - if (CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS - .equals(loadMetadataDetails.getLoadStatus)) { - LOGGER.info("DataLoad complete") - LOGGER.info("Data Load partially successful with LoadCount:" + loadCount) - } else { - LOGGER.info("DataLoad complete") - LOGGER.info("Data Loaded successfully with LoadCount:" + loadCount) - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.printStatisticsInfo( - model.getPartitionId) - } - } - } - } -} - -/** - * Use this RDD class to load csv data file - * - * @param sc The SparkContext to associate the RDD with. - * @param result Output result - * @param carbonLoadModel Carbon load model which contain the load info - * @param storePath The store location - * @param kettleHomePath The kettle home path - * @param columnar whether it is columnar - * @param loadCount Current load count - * @param tableCreationTime Time of creating table - * @param schemaLastUpdatedTime Time of last schema update - * @param blocksGroupBy Blocks Array which is group by partition or host - * @param isTableSplitPartition Whether using table split partition - * @tparam K Class of the key associated with the Result. - * @tparam V Class of the value associated with the Result. - */ -class DataFileLoaderRDD[K, V]( - sc: SparkContext, - result: DataLoadResult[K, V], - carbonLoadModel: CarbonLoadModel, - storePath: String, - kettleHomePath: String, - columnar: Boolean, - loadCount: Integer, - tableCreationTime: Long, - schemaLastUpdatedTime: Long, - blocksGroupBy: Array[(String, Array[BlockDetails])], - isTableSplitPartition: Boolean) extends RDD[(K, V)](sc, Nil) { - - sc.setLocalProperty("spark.scheduler.pool", "DDL") - - override def getPartitions: Array[Partition] = { - if (isTableSplitPartition) { - // for table split partition - var splits = Array[TableSplit]() - if (carbonLoadModel.isDirectLoad) { - splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath) - } else { - splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName, - carbonLoadModel.getTableName, null) - } - - splits.zipWithIndex.map { case (split, index) => - // filter the same partition unique id, because only one will match, so get 0 element - val blocksDetails: Array[BlockDetails] = blocksGroupBy.filter { case (uniqueId, _) => - uniqueId == split.getPartition.getUniqueID - }(0)._2 - new CarbonTableSplitPartition(id, index, split, blocksDetails) - } - } else { - // for node partition - blocksGroupBy.zipWithIndex.map { case ((uniqueId, blockDetails), index) => - new CarbonNodePartition(id, index, uniqueId, blockDetails) - } - } - } - - override def checkpoint() { - // Do nothing. Hadoop RDD should not be checkpointed. - } - - override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = { - val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) - val iter = new Iterator[(K, V)] { - var partitionID = "0" - val loadMetadataDetails = new LoadMetadataDetails() - var model: CarbonLoadModel = _ - val uniqueLoadStatusId = carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + - theSplit.index - try { - loadMetadataDetails.setPartitionCount(partitionID) - carbonLoadModel.setSegmentId(String.valueOf(loadCount)) - setModelAndBlocksInfo() - val loader = new SparkPartitionLoader(model, theSplit.index, storePath, - kettleHomePath, String.valueOf(loadCount), loadMetadataDetails) - loader.initialize - if (model.isRetentionRequest) { - recreateAggregationTableForRetention - } else if (model.isAggLoadRequest) { - loadMetadataDetails.setLoadStatus(createManualAggregateTable) - } else { - loader.run() - } - } catch { - case e: Exception => - logInfo("DataLoad failure") - LOGGER.error(e) - throw e - } - - def setModelAndBlocksInfo(): Unit = { - if (isTableSplitPartition) { - // for table split partition - val split = theSplit.asInstanceOf[CarbonTableSplitPartition] - logInfo("Input split: " + split.serializableHadoopSplit.value) - val blocksID = gernerateBlocksID - carbonLoadModel.setBlocksID(blocksID) - carbonLoadModel.setTaskNo(String.valueOf(theSplit.index)) - if (carbonLoadModel.isDirectLoad) { - model = carbonLoadModel.getCopyWithPartition( - split.serializableHadoopSplit.value.getPartition.getUniqueID, - split.serializableHadoopSplit.value.getPartition.getFilesPath, - carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter) - } else { - model = carbonLoadModel.getCopyWithPartition( - split.serializableHadoopSplit.value.getPartition.getUniqueID) - } - partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID - // get this partition data blocks and put it to global static map - GraphGenerator.blockInfo.put(blocksID, split.partitionBlocksDetail) - StandardLogService.setThreadName(partitionID, null) - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordPartitionBlockMap( - partitionID, split.partitionBlocksDetail.length) - } else { - // for node partition - val split = theSplit.asInstanceOf[CarbonNodePartition] - logInfo("Input split: " + split.serializableHadoopSplit) - logInfo("The Block Count in this node: " + split.nodeBlocksDetail.length) - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap( - split.serializableHadoopSplit, split.nodeBlocksDetail.length) - val blocksID = gernerateBlocksID - carbonLoadModel.setBlocksID(blocksID) - carbonLoadModel.setTaskNo(String.valueOf(theSplit.index)) - // set this node blocks info to global static map - GraphGenerator.blockInfo.put(blocksID, split.nodeBlocksDetail) - if (carbonLoadModel.isDirectLoad) { - val filelist: java.util.List[String] = new java.util.ArrayList[String]( - CarbonCommonConstants.CONSTANT_SIZE_TEN) - CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",") - model = carbonLoadModel.getCopyWithPartition(partitionID, filelist, - carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter) - } else { - model = carbonLoadModel.getCopyWithPartition(partitionID) - } - StandardLogService.setThreadName(blocksID, null) - } - } - - /** - * generate blocks id - * - * @return - */ - def gernerateBlocksID: String = { - if (isTableSplitPartition) { - carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" + - theSplit.asInstanceOf[CarbonTableSplitPartition].serializableHadoopSplit.value - .getPartition.getUniqueID + "_" + UUID.randomUUID() - } else { - carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" + - UUID.randomUUID() - } - } - - def checkAndLoadAggregationTable: String = { - val schema = model.getCarbonDataLoadSchema - val aggTables = schema.getCarbonTable.getAggregateTablesName - if (null != aggTables && !aggTables.isEmpty) { - val details = model.getLoadMetadataDetails.asScala.toArray - val newSlice = CarbonCommonConstants.LOAD_FOLDER + loadCount - var listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details) - listOfLoadFolders = CarbonLoaderUtil.addNewSliceNameToList(newSlice, listOfLoadFolders) - val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details) - var listOfAllLoadFolders = CarbonQueryUtil.getListOfSlices(details) - listOfAllLoadFolders = CarbonLoaderUtil - .addNewSliceNameToList(newSlice, listOfAllLoadFolders) - val copyListOfLoadFolders = listOfLoadFolders.asScala.toList - val copyListOfUpdatedLoadFolders = listOfUpdatedLoadFolders.asScala.toList - loadTableSlices(listOfAllLoadFolders, details) - val loadFolders = Array[String]() - loadMetadataDetails.setLoadStatus(iterateOverAggTables(aggTables, - copyListOfLoadFolders.asJava, copyListOfUpdatedLoadFolders.asJava, loadFolders)) - if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals( - loadMetadataDetails.getLoadStatus)) { - // remove the current slice from memory not the table - CarbonLoaderUtil - .removeSliceFromMemory(model.getDatabaseName, model.getTableName, newSlice) - logInfo(s"Aggregate table creation failed") - } else { - logInfo("Aggregate tables creation successfull") - } - } - loadMetadataDetails.getLoadStatus - } - - def loadTableSlices(listOfAllLoadFolders: java.util.List[String], - loadMetadataDetails: Array[LoadMetadataDetails]) = { - CarbonProperties.getInstance().addProperty("carbon.cache.used", "false") - // TODO: Implement it - } - - def createManualAggregateTable: String = { - val details = model.getLoadMetadataDetails.asScala.toArray - val listOfAllLoadFolders = CarbonQueryUtil.getListOfSlices(details) - val listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details) - val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details) - loadTableSlices(listOfAllLoadFolders, details) - val loadFolders = Array[String]() - val aggTable = model.getAggTableName - loadMetadataDetails.setLoadStatus(loadAggregationTable(listOfLoadFolders, - listOfUpdatedLoadFolders, loadFolders)) - if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals( - loadMetadataDetails.getLoadStatus)) { - logInfo(s"Aggregate table creation failed :: $aggTable") - } else { - logInfo(s"Aggregate table creation successfull :: $aggTable") - } - loadMetadataDetails.getLoadStatus - } - - def recreateAggregationTableForRetention = { - val schema = model.getCarbonDataLoadSchema - val aggTables = schema.getCarbonTable.getAggregateTablesName - if (null != aggTables && !aggTables.isEmpty) { - val details = model.getLoadMetadataDetails.asScala.toArray - val listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details) - val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details) - val listOfAllLoadFolder = CarbonQueryUtil.getListOfSlices(details) - loadTableSlices(listOfAllLoadFolder, details) - val loadFolders = Array[String]() - iterateOverAggTables(aggTables, listOfLoadFolders, listOfUpdatedLoadFolders, loadFolders) - } - } - - // TODO Aggregate table needs to be handled - def iterateOverAggTables(aggTables: java.util.List[String], - listOfLoadFolders: java.util.List[String], - listOfUpdatedLoadFolders: java.util.List[String], - loadFolders: Array[String]): String = { - model.setAggLoadRequest(true) - aggTables.asScala.foreach { aggTable => - model.setAggTableName(aggTable) - loadMetadataDetails.setLoadStatus(loadAggregationTable(listOfLoadFolders, - listOfUpdatedLoadFolders, loadFolders)) - if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals( - loadMetadataDetails.getLoadStatus)) { - logInfo(s"Aggregate table creation failed :: aggTable") - return loadMetadataDetails.getLoadStatus - } - } - loadMetadataDetails.getLoadStatus - } - - def loadAggregationTable(listOfLoadFolders: java.util.List[String], - listOfUpdatedLoadFolders: java.util.List[String], - loadFolders: Array[String]): String = { - // TODO: Implement it - loadMetadataDetails.getLoadStatus - } - - var finished = false - - override def hasNext: Boolean = { - !finished - } - - override def next(): (K, V) = { - finished = true - result.getKey(uniqueLoadStatusId, loadMetadataDetails) - } - } - iter - } - - override def getPreferredLocations(split: Partition): Seq[String] = { - if (isTableSplitPartition) { - // for table split partition - val theSplit = split.asInstanceOf[CarbonTableSplitPartition] - val location = theSplit.serializableHadoopSplit.value.getLocations.asScala - location - } else { - // for node partition - val theSplit = split.asInstanceOf[CarbonNodePartition] - val firstOptionLocation: Seq[String] = List(theSplit.serializableHadoopSplit) - logInfo("Preferred Location for split: " + firstOptionLocation.head) - val blockMap = new util.LinkedHashMap[String, Integer]() - val tableBlocks = theSplit.blocksDetails - tableBlocks.foreach { tableBlock => - tableBlock.getLocations.foreach { location => - if (!firstOptionLocation.exists(location.equalsIgnoreCase)) { - val currentCount = blockMap.get(location) - if (currentCount == null) { - blockMap.put(location, 1) - } else { - blockMap.put(location, currentCount + 1) - } - } - } - } - - val sortedList = blockMap.entrySet().asScala.toSeq.sortWith((nodeCount1, nodeCount2) => { - nodeCount1.getValue > nodeCount2.getValue - } - ) - - val sortedNodesList = sortedList.map(nodeCount => nodeCount.getKey).take(2) - firstOptionLocation ++ sortedNodesList - } - } - -} - -/** - * Use this RDD class to load RDD - * - * @param sc - * @param result - * @param carbonLoadModel - * @param storePath - * @param kettleHomePath - * @param columnar - * @param loadCount - * @param tableCreationTime - * @param schemaLastUpdatedTime - * @param prev - * @tparam K - * @tparam V - */ -class DataFrameLoaderRDD[K, V]( - sc: SparkContext, - result: DataLoadResult[K, V], - carbonLoadModel: CarbonLoadModel, - storePath: String, - kettleHomePath: String, - columnar: Boolean, - loadCount: Integer, - tableCreationTime: Long, - schemaLastUpdatedTime: Long, - prev: DataLoadCoalescedRDD[Row]) extends RDD[(K, V)](prev) { - - sc.setLocalProperty("spark.scheduler.pool", "DDL") - - @DeveloperApi - override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = { - val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) - val resultIter = new Iterator[(K, V)] { - val partitionID = "0" - val loadMetadataDetails = new LoadMetadataDetails() - val uniqueLoadStatusId = carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + - theSplit.index - try { - loadMetadataDetails.setPartitionCount(partitionID) - carbonLoadModel.setPartitionId(partitionID) - carbonLoadModel.setSegmentId(String.valueOf(loadCount)) - carbonLoadModel.setTaskNo(String.valueOf(theSplit.index)) - val loader = new SparkPartitionLoader(carbonLoadModel, theSplit.index, storePath, - kettleHomePath, String.valueOf(loadCount), loadMetadataDetails) - loader.initialize - val rddIteratorKey = UUID.randomUUID().toString - try { - RddInputUtils.put(rddIteratorKey, - new PartitionIterator( - firstParent[DataLoadPartitionWrap[Row]].iterator(theSplit, context), - carbonLoadModel, - context)) - carbonLoadModel.setRddIteratorKey(rddIteratorKey) - loader.run() - } finally { - RddInputUtils.remove(rddIteratorKey) - } - } catch { - case e: Exception => - logInfo("DataLoad failure") - LOGGER.error(e) - throw e - } - - var finished = false - - override def hasNext: Boolean = !finished - - override def next(): (K, V) = { - finished = true - result.getKey(uniqueLoadStatusId, loadMetadataDetails) - } - } - resultIter - } - - override protected def getPartitions: Array[Partition] = firstParent[Row].partitions -} - -class PartitionIterator(partitionIter: Iterator[DataLoadPartitionWrap[Row]], - carbonLoadModel: CarbonLoadModel, - context: TaskContext) extends CarbonIterator[CarbonIterator[Array[String]]] { - val serializer = SparkEnv.get.closureSerializer.newInstance() - var serializeBuffer: ByteBuffer = null - def hasNext: Boolean = partitionIter.hasNext - - def next: CarbonIterator[Array[String]] = { - val value = partitionIter.next - // The rdd (which come from Hive Table) don't support to read dataframe concurrently. - // So here will create different rdd instance for each thread. - val newInstance = { - if (serializeBuffer == null) { - serializeBuffer = serializer.serialize[RDD[Row]](value.rdd) - } - serializeBuffer.rewind() - serializer.deserialize[RDD[Row]](serializeBuffer) - } - new RddIterator(newInstance.iterator(value.partition, context), - carbonLoadModel, - context) - } - override def initialize: Unit = { - SparkUtil.setTaskContext(context) - } -} -/** - * This class wrap Scala's Iterator to Java's Iterator. - * It also convert all columns to string data to use csv data loading flow. - * - * @param rddIter - * @param carbonLoadModel - * @param context - */ -class RddIterator(rddIter: Iterator[Row], - carbonLoadModel: CarbonLoadModel, - context: TaskContext) extends CarbonIterator[Array[String]] { - - val timeStampformatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants - .CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT) - val timeStampFormat = new SimpleDateFormat(timeStampformatString) - val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants - .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT) - val dateFormat = new SimpleDateFormat(dateFormatString) - val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1 - val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2 - val serializationNullFormat = - carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1) - def hasNext: Boolean = rddIter.hasNext - - def next: Array[String] = { - val row = rddIter.next() - val columns = new Array[String](row.length) - for (i <- 0 until columns.length) { - columns(i) = CarbonScalaUtil.getString(row.get(i), serializationNullFormat, - delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat) - } - columns - } - - override def initialize: Unit = { - SparkUtil.setTaskContext(context) - } - -} - -class RddIteratorForUpdate(rddIter: Iterator[Row], - carbonLoadModel: CarbonLoadModel) extends java.util.Iterator[Array[String]] { - val timeStampformatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants - .CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT) - val timeStampFormat = new SimpleDateFormat(timeStampformatString) - val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants - .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT) - val dateFormat = new SimpleDateFormat(dateFormatString) - val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1 - val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2 - val serializationNullFormat = - carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1) - - def hasNext: Boolean = rddIter.hasNext - - def next: Array[String] = { - val row = rddIter.next() - val columns = new Array[String](row.length) - for (i <- 0 until row.length) { - // columns(i) = CarbonScalaUtil.getStringForUpdate(row(i), delimiterLevel1, delimiterLevel2) - columns(i) = CarbonScalaUtil.getString(row.get(i), serializationNullFormat, - delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat) - if (columns(i).length() > CarbonCommonConstants.DEFAULT_COLUMN_LENGTH) { - sys.error(s" Error processing input: Length of parsed input (${ - CarbonCommonConstants - .DEFAULT_COLUMN_LENGTH - }) exceeds the maximum number of characters defined" - ) - } - } - columns - } - - def remove(): Unit = { - } -} - -object CarbonDataLoadForUpdate { - def initialize(model: CarbonLoadModel, - splitIndex: Int): String = { - val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null) - if (null == carbonPropertiesFilePath) { - System.setProperty("carbon.properties.filepath", - System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties") - } - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId) - CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true") - CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1") - CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true") - CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true") - CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true") - CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000") - CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false") - CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000") - - // this property is used to determine whether temp location for carbon is inside - // container temp dir or is yarn application directory. - val carbonUseLocalDir = CarbonProperties.getInstance() - .getProperty("carbon.use.local.dir", "false") - var storeLocation = "" - if(carbonUseLocalDir.equalsIgnoreCase("true")) { - val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf) - if (null != storeLocations && storeLocations.nonEmpty) { - storeLocation = storeLocations(Random.nextInt(storeLocations.length)) - } - if (storeLocation == null) { - storeLocation = System.getProperty("java.io.tmpdir") - } - } - else { - storeLocation = System.getProperty("java.io.tmpdir") - } - storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex - storeLocation - } - - def run(model: CarbonLoadModel, - index: Int, - hdfsStoreLocation: String, - kettleHomePath: String, - loadCount: String, - loadMetadataDetails: LoadMetadataDetails, - executorErrors: ExecutionErrors): Unit = { - val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) - try { - var storeLocation = "" - val carbonUseLocalDir = CarbonProperties.getInstance() - .getProperty("carbon.use.local.dir", "false") - if(carbonUseLocalDir.equalsIgnoreCase("true")) { - val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf) - if (null != storeLocations && storeLocations.nonEmpty) { - storeLocation = storeLocations(Random.nextInt(storeLocations.length)) - } - if (storeLocation == null) { - storeLocation = System.getProperty("java.io.tmpdir") - } - } - else { - storeLocation = System.getProperty("java.io.tmpdir") - } - storeLocation = storeLocation + '/' + System.nanoTime() + '/' + index - - CarbonLoaderUtil.executeGraph(model, storeLocation, hdfsStoreLocation, - kettleHomePath) - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS) - } catch { - case e: DataLoadingException => if (e.getErrorCode == - DataProcessorConstants.BAD_REC_FOUND) { - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS) - LOGGER.info("Bad Record Found") - } else if (e.getErrorCode == DataProcessorConstants.BAD_REC_FAILURE_ERROR_CODE) { - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) - executorErrors.failureCauses = FailureCauses.BAD_RECORDS - executorErrors.errorMsg = e.getMessage - } else { - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) - throw e - } - case e: Exception => - // this will be in case of any other exception where the executor has to rethrow and retry. - throw e - } finally { - // delete temp location data - try { - val isCompaction = false - CarbonLoaderUtil.deleteLocalDataLoadFolderLocation(model, isCompaction) - } catch { - case e: Exception => - LOGGER.error("Failed to delete local data" + e) - } - if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals( - loadMetadataDetails.getLoadStatus)) { - CarbonTimeStatisticsFactory.getLoadStatisticsInstance.printStatisticsInfo( - model.getPartitionId) - } - } - } -} - diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala index d24d29af147..ab0d603d854 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala @@ -220,9 +220,6 @@ class CarbonScanRDD( } if (!finished && !havePair) { finished = !reader.nextKeyValue - if (finished) { - reader.close() - } havePair = !finished } !finished diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala index 84c8ccf7321..3b38028ebd6 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala @@ -40,7 +40,6 @@ object Compactor { val storePath = compactionCallableModel.storePath val storeLocation = compactionCallableModel.storeLocation val carbonTable = compactionCallableModel.carbonTable - val kettleHomePath = compactionCallableModel.kettleHomePath val cubeCreationTime = compactionCallableModel.cubeCreationTime val loadsToMerge = compactionCallableModel.loadsToMerge val sc = compactionCallableModel.sqlContext @@ -59,7 +58,6 @@ object Compactor { storePath, carbonTable.getMetaDataFilepath, mergedLoadName, - kettleHomePath, cubeCreationTime, databaseName, factTableName, diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala index 3b3bac396ac..0ba99a86712 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala @@ -150,7 +150,6 @@ object DataManagementFunc { compactionModel: CompactionModel, executor: ExecutorService, sqlContext: SQLContext, - kettleHomePath: String, storeLocation: String): Unit = { val sortedSegments: util.List[LoadMetadataDetails] = new util.ArrayList[LoadMetadataDetails]( carbonLoadModel.getLoadMetadataDetails @@ -181,7 +180,6 @@ object DataManagementFunc { storePath, sqlContext, compactionModel, - kettleHomePath, carbonLoadModel, storeLocation ) @@ -239,7 +237,6 @@ object DataManagementFunc { storePath: String, sqlContext: SQLContext, compactionModel: CompactionModel, - kettleHomePath: String, carbonLoadModel: CarbonLoadModel, storeLocation: String): Unit = { @@ -252,7 +249,6 @@ object DataManagementFunc { carbonLoadModel, storeLocation, compactionModel.carbonTable, - kettleHomePath, compactionModel.tableCreationTime, loadsToMerge, sqlContext, diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala index cb2bd3ef6a5..50894d40a8f 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala @@ -25,12 +25,13 @@ import java.util.{Date, UUID} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.util.Random import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType} import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.{Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext} import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionWrap, RDD} import org.apache.spark.sql.Row import org.apache.spark.util.SparkUtil @@ -41,13 +42,14 @@ import org.apache.carbondata.common.logging.impl.StandardLogService import org.apache.carbondata.core.constants.CarbonCommonConstants import org.apache.carbondata.core.statusmanager.LoadMetadataDetails import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory} +import org.apache.carbondata.processing.csvload.BlockDetails import org.apache.carbondata.processing.csvload.CSVInputFormat import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator -import org.apache.carbondata.processing.csvreaderstep.BlockDetails import org.apache.carbondata.processing.model.CarbonLoadModel import org.apache.carbondata.processing.newflow.DataLoadExecutor import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException import org.apache.carbondata.spark.DataLoadResult +import org.apache.carbondata.spark.load.CarbonLoaderUtil import org.apache.carbondata.spark.splits.TableSplit import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil, CommonUtil} @@ -84,6 +86,81 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser } } +/** + * This partition class use to split by Host + * + */ +class CarbonNodePartition(rddId: Int, val idx: Int, host: String, + val blocksDetails: Array[BlockDetails]) + extends Partition { + + override val index: Int = idx + val serializableHadoopSplit = host + val nodeBlocksDetail = blocksDetails + + override def hashCode(): Int = 41 * (41 + rddId) + idx +} + +/** + * This partition class use to split by TableSplit + * + */ +class CarbonTableSplitPartition(rddId: Int, val idx: Int, @transient val tableSplit: TableSplit, + val blocksDetails: Array[BlockDetails]) + extends Partition { + + override val index: Int = idx + val serializableHadoopSplit = new SerializableWritable[TableSplit](tableSplit) + val partitionBlocksDetail = blocksDetails + + override def hashCode(): Int = 41 * (41 + rddId) + idx +} + +class SparkPartitionLoader(model: CarbonLoadModel, + splitIndex: Int, + storePath: String, + loadCount: String, + loadMetadataDetails: LoadMetadataDetails) { + private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName) + + var storeLocation: String = "" + + def initialize(): Unit = { + val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null) + if (null == carbonPropertiesFilePath) { + System.setProperty("carbon.properties.filepath", + System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties") + } + CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId) + CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true") + CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1") + CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true") + CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true") + CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true") + CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000") + CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false") + CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000") + + // this property is used to determine whether temp location for carbon is inside + // container temp dir or is yarn application directory. + val carbonUseLocalDir = CarbonProperties.getInstance() + .getProperty("carbon.use.local.dir", "false") + if (carbonUseLocalDir.equalsIgnoreCase("true")) { + val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf) + if (null != storeLocations && storeLocations.nonEmpty) { + storeLocation = storeLocations(Random.nextInt(storeLocations.length)) + } + if (storeLocation == null) { + storeLocation = System.getProperty("java.io.tmpdir") + } + } else { + storeLocation = System.getProperty("java.io.tmpdir") + } + storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex + } + +} + /** * It loads the data to carbon using @AbstractDataLoadProcessorStep */ @@ -157,7 +234,6 @@ class NewCarbonDataLoadRDD[K, V]( val loader = new SparkPartitionLoader(model, theSplit.index, null, - null, String.valueOf(loadCount), loadMetadataDetails) // Intialize to set carbon properties @@ -304,8 +380,7 @@ class NewCarbonDataLoadRDD[K, V]( /** * It loads the data to carbon from spark DataFrame using - * @see org.apache.carbondata.processing.newflow.DataLoadExecutor without - * kettle requirement + * @see org.apache.carbondata.processing.newflow.DataLoadExecutor */ class NewDataFrameLoaderRDD[K, V]( sc: SparkContext, @@ -355,7 +430,6 @@ class NewDataFrameLoaderRDD[K, V]( val loader = new SparkPartitionLoader(model, theSplit.index, null, - null, String.valueOf(loadCount), loadMetadataDetails) // Intialize to set carbon properties diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala index bb661b1391c..a36fb6364a1 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala @@ -31,11 +31,11 @@ import org.apache.carbondata.processing.newflow.DataLoadExecutor import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException /** - * Data load in case of update command with out kettle. + * Data load in case of update command . */ object UpdateDataLoad { - def DataLoadNoKettleForUpdate(segId: String, + def DataLoadForUpdate(segId: String, index: Int, iter: Iterator[Row], carbonLoadModel: CarbonLoadModel, @@ -50,7 +50,6 @@ object UpdateDataLoad { val loader = new SparkPartitionLoader(carbonLoadModel, index, null, - null, segId, loadMetadataDetails) // Intialize to set carbon properties diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala index 3a0e395f6fd..c95b7ea84ca 100644 --- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala +++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala @@ -93,58 +93,6 @@ object CarbonScalaUtil { } } - def getKettleHome(sqlContext: SQLContext): String = { - var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null) - if (null == kettleHomePath) { - kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home") - } - if (null == kettleHomePath) { - val carbonHome = System.getenv("CARBON_HOME") - if (null != carbonHome) { - kettleHomePath = carbonHome + "/processing/carbonplugins" - } - } - if (kettleHomePath != null) { - val sparkMaster = sqlContext.sparkContext.getConf.get("spark.master").toLowerCase() - // get spark master, if local, need to correct the kettle home - // e.g: --master local, the executor running in local machine - if (sparkMaster.startsWith("local")) { - val kettleHomeFileType = FileFactory.getFileType(kettleHomePath) - val kettleHomeFile = FileFactory.getCarbonFile(kettleHomePath, kettleHomeFileType) - // check if carbon.kettle.home path is exists - if (!kettleHomeFile.exists()) { - // get the path of this class - // e.g: file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/carbon- - // xxx.jar!/org/carbondata/spark/rdd/ - var jarFilePath = this.getClass.getResource("").getPath - val endIndex = jarFilePath.indexOf(".jar!") + 4 - // get the jar file path - // e.g: file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/carbon-*.jar - jarFilePath = jarFilePath.substring(0, endIndex) - val jarFileType = FileFactory.getFileType(jarFilePath) - val jarFile = FileFactory.getCarbonFile(jarFilePath, jarFileType) - // get the parent folder of the jar file - // e.g:file:/srv/bigdata/install/spark/sparkJdbc/carbonlib - val carbonLibPath = jarFile.getParentFile.getPath - // find the kettle home under the previous folder - // e.g:file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/cabonplugins - kettleHomePath = carbonLibPath + File.separator + CarbonCommonConstants.KETTLE_HOME_NAME - val logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName) - logger.error(s"carbon.kettle.home path is not exists, reset it as $kettleHomePath") - val newKettleHomeFileType = FileFactory.getFileType(kettleHomePath) - val newKettleHomeFile = FileFactory.getCarbonFile(kettleHomePath, newKettleHomeFileType) - // check if the found kettle home exists - if (!newKettleHomeFile.exists()) { - sys.error("Kettle home not found. Failed to reset carbon.kettle.home") - } - } - } - } else { - sys.error("carbon.kettle.home is not set") - } - kettleHomePath - } - def getString(value: Any, serializationNullFormat: String, delimiterLevel1: String, diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala index 9c5ce83a11d..81209421269 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala @@ -751,7 +751,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser { val supportedOptions = Seq("DELIMITER", "QUOTECHAR", "FILEHEADER", "ESCAPECHAR", "MULTILINE", "COMPLEX_DELIMITER_LEVEL_1", "COMPLEX_DELIMITER_LEVEL_2", "COLUMNDICT", "SERIALIZATION_NULL_FORMAT", "BAD_RECORDS_LOGGER_ENABLE", "BAD_RECORDS_ACTION", - "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "USE_KETTLE", "DATEFORMAT", + "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "DATEFORMAT", "SINGLE_PASS", "IS_EMPTY_DATA_BAD_RECORD" ) var isSupported = true diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala index c858d0a46f7..33467436d9a 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala @@ -86,7 +86,6 @@ case class CarbonMergerMapping(storeLocation: String, hdfsStoreLocation: String, metadataFilePath: String, var mergedLoadName: String, - kettleHomePath: String, tableCreationTime: Long, databaseName: String, factTableName: String, @@ -121,7 +120,6 @@ case class CompactionCallableModel(storePath: String, carbonLoadModel: CarbonLoadModel, storeLocation: String, carbonTable: CarbonTable, - kettleHomePath: String, cubeCreationTime: Long, loadsToMerge: util.List[LoadMetadataDetails], sqlContext: SQLContext, diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala index f8e2bc15f3d..a5fef5e0c45 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala @@ -48,7 +48,6 @@ object TestQueryExecutor { val storeLocation = s"$integrationPath/spark-common/target/store" val warehouse = s"$integrationPath/spark-common/target/warehouse" val metastoredb = s"$integrationPath/spark-common/target" - val kettleHome = s"$projectPath/processing/carbonplugins" val timestampFormat = "dd-MM-yyyy" val INSTANCE = lookupQueryExecutor.newInstance().asInstanceOf[TestQueryExecutorRegister] diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala index 7c6665c40b3..643002d5422 100644 --- a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala +++ b/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD} -import org.apache.carbondata.processing.csvreaderstep.BlockDetails +import org.apache.carbondata.processing.csvload.BlockDetails /* * this object use to handle file splits diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml index 1a288a8e4cc..6083fbaed0b 100644 --- a/integration/spark/pom.xml +++ b/integration/spark/pom.xml @@ -175,7 +175,6 @@ true - ${use.kettle} diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala index ebeeed206e6..0e2e4ddab29 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala @@ -186,19 +186,11 @@ class CarbonDataFrameWriter(val dataFrame: DataFrame) { } private def makeLoadString(csvFolder: String, options: CarbonOption): String = { - if (options.useKettle) { - s""" - LOAD DATA INPATH '$csvFolder' - INTO TABLE ${options.dbName}.${options.tableName} - OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', 'USE_KETTLE' = 'true') - """ - } else { - s""" + s""" LOAD DATA INPATH '$csvFolder' INTO TABLE ${options.dbName}.${options.tableName} - OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', 'USE_KETTLE' = 'false') + OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}') """ - } } diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala index dbd2a21c178..60742ac80c9 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala @@ -48,7 +48,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil import org.apache.carbondata.core.statusmanager.LoadMetadataDetails import org.apache.carbondata.core.util.CarbonProperties import org.apache.carbondata.core.util.path.CarbonStorePath -import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInpututilsForUpdate} +import org.apache.carbondata.processing.csvload.BlockDetails import org.apache.carbondata.processing.etl.DataLoadingException import org.apache.carbondata.processing.model.CarbonLoadModel import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException @@ -70,7 +70,6 @@ object CarbonDataRDDFactory { alterTableModel: AlterTableModel, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String): Unit = { var compactionSize: Long = 0 var compactionType: CompactionType = CompactionType.MINOR_COMPACTION @@ -130,7 +129,6 @@ object CarbonDataRDDFactory { handleCompactionForSystemLocking(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionType, carbonTable, @@ -150,7 +148,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -173,7 +170,6 @@ object CarbonDataRDDFactory { def handleCompactionForSystemLocking(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String, compactionType: CompactionType, carbonTable: CarbonTable, @@ -189,7 +185,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -226,7 +221,6 @@ object CarbonDataRDDFactory { def startCompactionThreads(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String, compactionModel: CompactionModel, compactionLock: ICarbonLock): Unit = { @@ -254,7 +248,7 @@ object CarbonDataRDDFactory { DataManagementFunc.executeCompaction(carbonLoadModel: CarbonLoadModel, storePath: String, compactionModel: CompactionModel, - executor, sqlContext, kettleHomePath, storeLocation + executor, sqlContext, storeLocation ) triggeredCompactionStatus = true } catch { @@ -303,7 +297,7 @@ object CarbonDataRDDFactory { DataManagementFunc.executeCompaction(newCarbonLoadModel, newCarbonLoadModel.getStorePath, newcompactionModel, - executor, sqlContext, kettleHomePath, storeLocation + executor, sqlContext, storeLocation ) } catch { case e: Exception => @@ -351,10 +345,8 @@ object CarbonDataRDDFactory { def loadCarbonData(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, columnar: Boolean, partitionStatus: String = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS, - useKettle: Boolean, result: Future[DictionaryServer], dataFrame: Option[DataFrame] = None, updateModel: Option[UpdateTableModel] = None): Unit = { @@ -396,7 +388,6 @@ object CarbonDataRDDFactory { handleCompactionForSystemLocking(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, CompactionType.MINOR_COMPACTION, carbonTable, @@ -414,7 +405,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -444,10 +434,6 @@ object CarbonDataRDDFactory { try { LOGGER.audit(s"Data load request has been received for table" + s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }") - if (!useKettle) { - LOGGER.audit("Data is loading with New Data Flow for table " + - s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }") - } // Check if any load need to be deleted before loading new data DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName, storePath, isForceDeletion = false) @@ -555,8 +541,7 @@ object CarbonDataRDDFactory { * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block, * for locally writing carbondata files(one file one block) in nodes - * 4)use kettle: use DataFileLoaderRDD to load data and write to carbondata files - * non kettle: use NewCarbonDataLoadRDD to load data and write to carbondata files + * use NewCarbonDataLoadRDD to load data and write to carbondata files */ val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) // FileUtils will skip file which is no csv, and return all file path which split by ',' @@ -621,27 +606,12 @@ object CarbonDataRDDFactory { ).toArray } - if (useKettle) { - status = new DataFileLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - storePath, - kettleHomePath, - columnar, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - blocksGroupBy, - isTableSplitPartition - ).collect() - } else { - status = new NewCarbonDataLoadRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - currentLoadCount, - blocksGroupBy, - isTableSplitPartition).collect() - } + status = new NewCarbonDataLoadRDD(sqlContext.sparkContext, + new DataLoadResultImpl(), + carbonLoadModel, + currentLoadCount, + blocksGroupBy, + isTableSplitPartition).collect() } def loadDataFrame(): Unit = { @@ -653,31 +623,17 @@ object CarbonDataRDDFactory { val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData, sqlContext.sparkContext) val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct) - if (useKettle) { - status = new DataFrameLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - storePath, - kettleHomePath, - columnar, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - newRdd).collect() - } else { + var numPartitions = DistributionUtil.getNodeList(sqlContext.sparkContext).length + numPartitions = Math.max(1, Math.min(numPartitions, rdd.partitions.length)) + val coalesceRdd = rdd.coalesce(numPartitions, shuffle = false) - var numPartitions = DistributionUtil.getNodeList(sqlContext.sparkContext).length - numPartitions = Math.max(1, Math.min(numPartitions, rdd.partitions.length)) - val coalesceRdd = rdd.coalesce(numPartitions, shuffle = false) - - status = new NewDataFrameLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - newRdd).collect() - } + status = new NewDataFrameLoaderRDD(sqlContext.sparkContext, + new DataLoadResultImpl(), + carbonLoadModel, + currentLoadCount, + tableCreationTime, + schemaLastUpdatedTime, + newRdd).collect() } catch { case ex: Exception => LOGGER.error(ex, "load data frame failed") @@ -726,29 +682,11 @@ object CarbonDataRDDFactory { loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS) val rddIteratorKey = CarbonCommonConstants.RDDUTIL_UPDATE_KEY + UUID.randomUUID().toString - if (useKettle) { - try { - RddInpututilsForUpdate.put(rddIteratorKey, - new RddIteratorForUpdate(iter, carbonLoadModel)) - carbonLoadModel.setRddIteratorKey(rddIteratorKey) - CarbonDataLoadForUpdate.run(carbonLoadModel, - index, - storePath, - kettleHomePath, - segId, - loadMetadataDetails, - executionErrors) - } finally { - RddInpututilsForUpdate.remove(rddIteratorKey) - } - } else { - UpdateDataLoad.DataLoadNoKettleForUpdate(segId, - index, - iter, - carbonLoadModel, - loadMetadataDetails) - - } + UpdateDataLoad.DataLoadForUpdate(segId, + index, + iter, + carbonLoadModel, + loadMetadataDetails) } catch { case e: Exception => LOGGER.info("DataLoad failure") diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala index 6f14febba97..5a22e9c3d10 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala @@ -110,8 +110,6 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName) carbonLoadModel.setStorePath(relation.tableMeta.storePath) - val kettleHomePath = CarbonScalaUtil.getKettleHome(sqlContext) - var storeLocation = CarbonProperties.getInstance .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH, System.getProperty("java.io.tmpdir") @@ -122,7 +120,6 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e alterTableModel, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, storeLocation ) } catch { @@ -387,28 +384,6 @@ case class LoadTable( val columnar = sqlContext.getConf("carbon.is.columnar.storage", "true").toBoolean - // TODO It will be removed after kettle is removed. - val useKettle = options.get("use_kettle") match { - case Some(value) => value.toBoolean - case _ => - var useKettleLocal = System.getProperty("use_kettle") - if (useKettleLocal == null && sqlContext.sparkContext.getConf.contains("use_kettle")) { - useKettleLocal = sqlContext.sparkContext.getConf.get("use_kettle") - } - if (useKettleLocal == null) { - useKettleLocal = CarbonProperties.getInstance(). - getProperty(CarbonCommonConstants.USE_KETTLE, - CarbonCommonConstants.USE_KETTLE_DEFAULT) - } - try { - useKettleLocal.toBoolean - } catch { - case e: Exception => CarbonCommonConstants.USE_KETTLE_DEFAULT.toBoolean - } - } - - val kettleHomePath = if (useKettle) CarbonScalaUtil.getKettleHome(sqlContext) else "" - val delimiter = options.getOrElse("delimiter", ",") val quoteChar = options.getOrElse("quotechar", "\"") val fileHeader = options.getOrElse("fileheader", "") @@ -452,14 +427,14 @@ case class LoadTable( carbonLoadModel .setIsEmptyDataBadRecord( DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + isEmptyDataBadRecord) - // when single_pass=true, and use_kettle=false, and not use all dict + // when single_pass=true, and not use all dict val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match { case "true" => - if (!useKettle && StringUtils.isEmpty(allDictionaryPath)) { + if (StringUtils.isEmpty(allDictionaryPath)) { true } else { LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH" + - "can not be used together, and USE_KETTLE must be set as false") + "can not be used together") false } case "false" => @@ -540,10 +515,8 @@ case class LoadTable( CarbonDataRDDFactory.loadCarbonData(sqlContext, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, columnar, partitionStatus, - useKettle, result, dataFrame, updateModel) @@ -587,10 +560,8 @@ case class LoadTable( CarbonDataRDDFactory.loadCarbonData(sqlContext, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, columnar, partitionStatus, - useKettle, result, loadDataFrame, updateModel) diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala b/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala index d9c6fbbaaac..591cdf43e5e 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala @@ -38,7 +38,6 @@ object SparkTestQueryExecutor { private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName) LOGGER.info("use TestQueryExecutorImplV1") CarbonProperties.getInstance() - .addProperty("carbon.kettle.home", TestQueryExecutor.kettleHome) .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, TestQueryExecutor.timestampFormat) .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH, System.getProperty("java.io.tmpdir")) @@ -47,8 +46,7 @@ object SparkTestQueryExecutor { val sc = new SparkContext(new SparkConf() .setAppName("CarbonSpark") .setMaster("local[2]") - .set("spark.sql.shuffle.partitions", "20") - .set("use_kettle_default", "true")) + .set("spark.sql.shuffle.partitions", "20")) sc.setLogLevel("ERROR") val cc = new CarbonContext(sc, TestQueryExecutor.storeLocation, TestQueryExecutor.metastoredb) diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala index 885a0f0c610..851f7e9d11d 100644 --- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala +++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala @@ -65,18 +65,6 @@ class BadRecordLoggerSharedDictionaryTest extends QueryTest with BeforeAndAfterA } test("dataload with bad record test") { - try { - sql( - s"""LOAD DATA INPATH '$csvFilePath' INTO TABLE testdrive OPTIONS('DELIMITER'=',', - |'QUOTECHAR'= '"', 'BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL', - |'FILEHEADER'= 'ID,CUST_ID,cust_name', 'USE_KETTLE' = 'TRUE')""".stripMargin) - } catch { - case e: Throwable => - assert(e.getMessage.contains("Data load failed due to bad record")) - } - } - - test("dataload with no kettle") { try { sql( s"""LOAD DATA INPATH '$csvFilePath' INTO TABLE testdrive OPTIONS('DELIMITER'=',', diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala index 8a9d8af9a39..1d456d3a147 100644 --- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala +++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala @@ -41,7 +41,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll { sql( s""" |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_two_pass - |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='false') + |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false') """.stripMargin) sql( @@ -54,7 +54,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll { sql( s""" |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass - |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true') + |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true') """.stripMargin) } @@ -75,7 +75,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll { sql( s""" |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass_2 - |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true', 'COLUMNDICT'= + |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true', 'COLUMNDICT'= |'country:$resourcesPath/columndictionary/country.csv, name:$resourcesPath/columndictionary/name.csv') """.stripMargin) @@ -89,12 +89,12 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll { sql( s""" |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_two_pass - |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='false') + |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false') """.stripMargin) sql( s""" |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_one_pass - |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true') + |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true') """.stripMargin) checkAnswer( diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml index 44bce5ea017..f82779243e0 100644 --- a/integration/spark2/pom.xml +++ b/integration/spark2/pom.xml @@ -156,7 +156,6 @@ true - ${use.kettle} diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala index 2fc918def4f..7cb5ed48160 100644 --- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala +++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala @@ -48,7 +48,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil import org.apache.carbondata.core.statusmanager.LoadMetadataDetails import org.apache.carbondata.core.util.CarbonProperties import org.apache.carbondata.core.util.path.CarbonStorePath -import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInpututilsForUpdate} +import org.apache.carbondata.processing.csvload.BlockDetails import org.apache.carbondata.processing.etl.DataLoadingException import org.apache.carbondata.processing.model.CarbonLoadModel import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException @@ -70,7 +70,6 @@ object CarbonDataRDDFactory { alterTableModel: AlterTableModel, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String): Unit = { var compactionSize: Long = 0 var compactionType: CompactionType = CompactionType.MINOR_COMPACTION @@ -136,7 +135,6 @@ object CarbonDataRDDFactory { handleCompactionForSystemLocking(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionType, carbonTable, @@ -156,7 +154,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -179,7 +176,6 @@ object CarbonDataRDDFactory { def handleCompactionForSystemLocking(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String, compactionType: CompactionType, carbonTable: CarbonTable, @@ -195,7 +191,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -232,7 +227,6 @@ object CarbonDataRDDFactory { def startCompactionThreads(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, storeLocation: String, compactionModel: CompactionModel, compactionLock: ICarbonLock): Unit = { @@ -264,7 +258,7 @@ object CarbonDataRDDFactory { DataManagementFunc.executeCompaction(carbonLoadModel: CarbonLoadModel, storePath: String, compactionModel: CompactionModel, - executor, sqlContext, kettleHomePath, storeLocation + executor, sqlContext, storeLocation ) triggeredCompactionStatus = true } catch { @@ -313,7 +307,7 @@ object CarbonDataRDDFactory { DataManagementFunc.executeCompaction(newCarbonLoadModel, newCarbonLoadModel.getStorePath, newcompactionModel, - executor, sqlContext, kettleHomePath, storeLocation + executor, sqlContext, storeLocation ) } catch { case e: Exception => @@ -361,10 +355,8 @@ object CarbonDataRDDFactory { def loadCarbonData(sqlContext: SQLContext, carbonLoadModel: CarbonLoadModel, storePath: String, - kettleHomePath: String, columnar: Boolean, partitionStatus: String = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS, - useKettle: Boolean, result: Future[DictionaryServer], dataFrame: Option[DataFrame] = None, updateModel: Option[UpdateTableModel] = None): Unit = { @@ -406,7 +398,6 @@ object CarbonDataRDDFactory { handleCompactionForSystemLocking(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, CompactionType.MINOR_COMPACTION, carbonTable, @@ -424,7 +415,6 @@ object CarbonDataRDDFactory { startCompactionThreads(sqlContext, carbonLoadModel, storePath, - kettleHomePath, storeLocation, compactionModel, lock @@ -454,10 +444,6 @@ object CarbonDataRDDFactory { try { LOGGER.audit(s"Data load request has been received for table" + s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }") - if (!useKettle) { - LOGGER.audit("Data is loading with New Data Flow for table " + - s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }") - } // Check if any load need to be deleted before loading new data DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName, storePath, isForceDeletion = false) @@ -565,8 +551,7 @@ object CarbonDataRDDFactory { * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block, * for locally writing carbondata files(one file one block) in nodes - * 4)use kettle: use DataFileLoaderRDD to load data and write to carbondata files - * non kettle: use NewCarbonDataLoadRDD to load data and write to carbondata files + * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files */ val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration) // FileUtils will skip file which is no csv, and return all file path which split by ',' @@ -631,27 +616,12 @@ object CarbonDataRDDFactory { ).toArray } - if (useKettle) { - status = new DataFileLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - storePath, - kettleHomePath, - columnar, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - blocksGroupBy, - isTableSplitPartition - ).collect() - } else { - status = new NewCarbonDataLoadRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - currentLoadCount, - blocksGroupBy, - isTableSplitPartition).collect() - } + status = new NewCarbonDataLoadRDD(sqlContext.sparkContext, + new DataLoadResultImpl(), + carbonLoadModel, + currentLoadCount, + blocksGroupBy, + isTableSplitPartition).collect() } def loadDataFrame(): Unit = { @@ -665,26 +635,13 @@ object CarbonDataRDDFactory { sqlContext.sparkContext) val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct) - if (useKettle) { - status = new DataFrameLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - storePath, - kettleHomePath, - columnar, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - newRdd).collect() - } else { - status = new NewDataFrameLoaderRDD(sqlContext.sparkContext, - new DataLoadResultImpl(), - carbonLoadModel, - currentLoadCount, - tableCreationTime, - schemaLastUpdatedTime, - newRdd).collect() - } + status = new NewDataFrameLoaderRDD(sqlContext.sparkContext, + new DataLoadResultImpl(), + carbonLoadModel, + currentLoadCount, + tableCreationTime, + schemaLastUpdatedTime, + newRdd).collect() } catch { case ex: Exception => @@ -693,103 +650,6 @@ object CarbonDataRDDFactory { } } - def loadDataFrameForUpdate(): Unit = { - def triggerDataLoadForSegment(key: String, - iter: Iterator[Row]): Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] = { - val rddResult = new updateResultImpl() - val LOGGER = LogServiceFactory.getLogService(this.getClass.getName) - val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] { - var partitionID = "0" - val loadMetadataDetails = new LoadMetadataDetails - val executionErrors = new ExecutionErrors(FailureCauses.NONE, "") - var uniqueLoadStatusId = "" - try { - val segId = key - val taskNo = CarbonUpdateUtil - .getLatestTaskIdForSegment(segId, - CarbonStorePath.getCarbonTablePath(carbonLoadModel.getStorePath, - carbonTable.getCarbonTableIdentifier)) - val index = taskNo + 1 - uniqueLoadStatusId = carbonLoadModel.getTableName + - CarbonCommonConstants.UNDERSCORE + - (index + "_0") - - // convert timestamp - val timeStampInLong = updateModel.get.updatedTimeStamp + "" - loadMetadataDetails.setPartitionCount(partitionID) - loadMetadataDetails.setLoadName(segId) - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) - carbonLoadModel.setPartitionId(partitionID) - carbonLoadModel.setSegmentId(segId) - carbonLoadModel.setTaskNo(String.valueOf(index)) - carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp) - - // During Block Spill case Increment of File Count and proper adjustment of Block - // naming is only done when AbstractFactDataWriter.java : initializeWriter get - // CarbondataFileName as null. For handling Block Spill not setting the - // CarbondataFileName in case of Update. - // carbonLoadModel.setCarbondataFileName(newBlockName) - - // storeLocation = CarbonDataLoadRDD.initialize(carbonLoadModel, index) - loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS) - val rddIteratorKey = CarbonCommonConstants.RDDUTIL_UPDATE_KEY + - UUID.randomUUID().toString - - try { - RddInpututilsForUpdate.put(rddIteratorKey, - new RddIteratorForUpdate(iter, carbonLoadModel)) - carbonLoadModel.setRddIteratorKey(rddIteratorKey) - CarbonDataLoadForUpdate - .run(carbonLoadModel, index, storePath, kettleHomePath, - segId, loadMetadataDetails, executionErrors) - } finally { - RddInpututilsForUpdate.remove(rddIteratorKey) - } - } catch { - case e: Exception => - LOGGER.info("DataLoad failure") - LOGGER.error(e) - throw e - } - - var finished = false - - override def hasNext: Boolean = !finished - - override def next(): (String, (LoadMetadataDetails, ExecutionErrors)) = { - finished = true - rddResult - .getKey(uniqueLoadStatusId, - (loadMetadataDetails, executionErrors)) - } - } - resultIter - } - - val updateRdd = dataFrame.get.rdd - - - val keyRDD = updateRdd.map(row => - // splitting as (key, value) i.e., (segment, updatedRows) - (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)) - ) - val groupBySegmentRdd = keyRDD.groupByKey() - - val nodeNumOfData = groupBySegmentRdd.partitions.flatMap[String, Array[String]] { p => - DataLoadPartitionCoalescer.getPreferredLocs(groupBySegmentRdd, p).map(_.host) - }.distinct.size - val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData, - sqlContext.sparkContext) - val groupBySegmentAndNodeRdd = - new UpdateCoalescedRDD[(String, scala.Iterable[Row])](groupBySegmentRdd, - nodes.distinct.toArray) - - res = groupBySegmentAndNodeRdd.map(x => - triggerDataLoadForSegment(x._1, x._2.toIterator).toList - ).collect() - - } - if (!updateModel.isDefined) { CarbonLoaderUtil.checkAndCreateCarbonDataLocation(storePath, carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName, currentLoadCount.toString) @@ -798,9 +658,7 @@ object CarbonDataRDDFactory { var errorMessage: String = "DataLoad failure" var executorMessage: String = "" try { - if (updateModel.isDefined) { - loadDataFrameForUpdate() - } else if (dataFrame.isDefined) { + if (dataFrame.isDefined) { loadDataFrame() } else { diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala index 63cf9de1ddf..e6efeaa7aec 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala @@ -190,7 +190,6 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) { | LOAD DATA INPATH '$csvFolder' | INTO TABLE ${options.dbName}.${options.tableName} | OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', - | 'USE_KETTLE' = '${options.useKettle}', | 'SINGLE_PASS' = '${options.singlePass}') """.stripMargin } diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala index 002b6f89371..4bd0564b967 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala @@ -110,8 +110,6 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName) carbonLoadModel.setStorePath(relation.tableMeta.storePath) - val kettleHomePath = CarbonScalaUtil.getKettleHome(sparkSession.sqlContext) - var storeLocation = CarbonProperties.getInstance .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH, System.getProperty("java.io.tmpdir") @@ -123,7 +121,6 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab alterTableModel, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, storeLocation ) } catch { @@ -388,29 +385,6 @@ case class LoadTable( val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean - // TODO It will be removed after kettle is removed. - val useKettle = options.get("use_kettle") match { - case Some(value) => value.toBoolean - case _ => - var useKettleLocal = System.getProperty("use_kettle") - if (useKettleLocal == null && sparkSession.sparkContext.getConf.contains("use_kettle")) { - useKettleLocal = sparkSession.sparkContext.getConf.get("use_kettle") - } - if (useKettleLocal == null) { - useKettleLocal = CarbonProperties.getInstance(). - getProperty(CarbonCommonConstants.USE_KETTLE, - CarbonCommonConstants.USE_KETTLE_DEFAULT) - } - try { - useKettleLocal.toBoolean - } catch { - case e: Exception => CarbonCommonConstants.USE_KETTLE_DEFAULT.toBoolean - } - } - - val kettleHomePath = - if (useKettle) CarbonScalaUtil.getKettleHome(sparkSession.sqlContext) else "" - val delimiter = options.getOrElse("delimiter", ",") val quoteChar = options.getOrElse("quotechar", "\"") val fileHeader = options.getOrElse("fileheader", "") @@ -455,11 +429,11 @@ case class LoadTable( DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + isEmptyDataBadRecord) val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match { case "true" => - if (!useKettle && StringUtils.isEmpty(allDictionaryPath)) { + if (StringUtils.isEmpty(allDictionaryPath)) { true } else { LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH" + - "can not be used together, and USE_KETTLE must be set as false") + "can not be used together") false } case "false" => @@ -539,10 +513,8 @@ case class LoadTable( CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, columnar, partitionStatus, - useKettle, result, dataFrame, updateModel) @@ -590,10 +562,8 @@ case class LoadTable( CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext, carbonLoadModel, relation.tableMeta.storePath, - kettleHomePath, columnar, partitionStatus, - useKettle, result, loadDataFrame, updateModel) diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala index 15d5597e8eb..e94b6ed409e 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala @@ -38,7 +38,6 @@ object Spark2TestQueryExecutor { private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName) LOGGER.info("use TestQueryExecutorImplV2") CarbonProperties.getInstance() - .addProperty("carbon.kettle.home", TestQueryExecutor.kettleHome) .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, TestQueryExecutor.timestampFormat) .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH, System.getProperty("java.io.tmpdir")) diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala index bcc82ceae57..324d3a256df 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala @@ -78,12 +78,6 @@ object TableLoader { System.out.println(s"table name: $dbName.$tableName") val inputPaths = TableAPIUtil.escape(args(2)) - val kettleHome = CarbonProperties.getInstance().getProperty("carbon.kettle.home") - if (kettleHome == null) { - CarbonProperties.getInstance().addProperty("carbon.kettle.home", - map.getOrElse("carbon.kettle.home", "")) - } - val spark = TableAPIUtil.spark(storePath, s"TableLoader: $dbName.$tableName") CarbonEnv.init(spark) diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala index 33f710b9a39..642b3304985 100644 --- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala +++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala @@ -54,7 +54,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t4") """) LoadTable(Some("default"), "t4", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default_t4") if (table != null && table.getBucketingInfo("t4") != null) { assert(true) @@ -90,7 +90,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("tableName"="t5") """) LoadTable(Some("default"), "t5", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) val plan = sql( """ @@ -115,7 +115,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t6") """) LoadTable(Some("default"), "t6", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) val plan = sql( """ @@ -140,7 +140,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t7") """) LoadTable(Some("default"), "t7", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) sql("DROP TABLE IF EXISTS bucketed_parquet_table") sql("select * from t7").write @@ -171,7 +171,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t8") """) LoadTable(Some("default"), "t8", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) sql("DROP TABLE IF EXISTS parquet_table") sql("select * from t8").write diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala index ceb340e6576..55eaa20a0bf 100644 --- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala +++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala @@ -44,7 +44,7 @@ class VectorReaderTestCase extends QueryTest with BeforeAndAfterAll { OPTIONS("tableName"="vectorreader") """) LoadTable(Some("default"), "vectorreader", s"$resourcesPath/source.csv", Nil, - Map(("use_kettle", "false"))).run(sqlContext.sparkSession) + Map()).run(sqlContext.sparkSession) } test("test vector reader") { diff --git a/pom.xml b/pom.xml index 95a3ff4d1d2..a132d6dabb0 100644 --- a/pom.xml +++ b/pom.xml @@ -105,8 +105,6 @@ UTF-8 1.1.2.6 2.2.0 - 4.4.0-stable - false compile compile compile diff --git a/processing/carbonplugins/.kettle/kettle.properties b/processing/carbonplugins/.kettle/kettle.properties deleted file mode 100644 index 9bae2e3188d..00000000000 --- a/processing/carbonplugins/.kettle/kettle.properties +++ /dev/null @@ -1,10 +0,0 @@ -# This file was generated by Pentaho Data Integration version 4.2.1. -# -# Here are a few examples of variables to set: -# -# PRODUCTION_SERVER = hercules -# TEST_SERVER = zeus -# DEVELOPMENT_SERVER = thor -# -# Note: lines like these with a # in front of it are comments -# diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml deleted file mode 100644 index 2ae5a878a0d..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml deleted file mode 100644 index fc3cdf59d41..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml deleted file mode 100644 index a0a21923ae9..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml deleted file mode 100644 index 91617c57d12..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml deleted file mode 100644 index 7f33b5e5119..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml deleted file mode 100644 index 71731711b2d..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml +++ /dev/null @@ -1,29 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml deleted file mode 100644 index 92631a1633c..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml deleted file mode 100644 index 270bbe35879..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml deleted file mode 100644 index 087589e8327..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml deleted file mode 100644 index 550531cf9d9..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml deleted file mode 100644 index 517b757ad25..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml deleted file mode 100644 index 618dba08c04..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml deleted file mode 100644 index 970a855260a..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml deleted file mode 100644 index e0892a4ebc7..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml +++ /dev/null @@ -1,28 +0,0 @@ - - - - - - diff --git a/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml deleted file mode 100644 index d08d0ee8166..00000000000 --- a/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - - diff --git a/processing/pom.xml b/processing/pom.xml index 096f49a0a46..57b090845cc 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -48,37 +48,11 @@ org.apache.spark spark-sql_${scala.binary.version} - - - pentaho-kettle - kettle-engine - ${kettle.version} com.univocity univocity-parsers 1.5.6 - - - pentaho-kettle - kettle-core - ${kettle.version} - - - pentaho-kettle - kettle-db - ${kettle.version} - - - commons-vfs - commons-vfs - 1.0 - - - * - * - - org.apache.commons diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java similarity index 95% rename from processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java rename to processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java index 328c3b74967..d6d214bba2c 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java +++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.processing.csvreaderstep; +package org.apache.carbondata.processing.csvload; import java.io.Serializable; @@ -26,7 +26,6 @@ /** * blocks info - * TODO Remove this class after removing of kettle. */ public class BlockDetails extends FileSplit implements Serializable { diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java deleted file mode 100644 index 7e6f6f47d64..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java +++ /dev/null @@ -1,475 +0,0 @@ -/* - * 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.csvload; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.filesystem.CarbonFile; -import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.datastore.impl.FileFactory.FileType; -import org.apache.carbondata.processing.api.dataloader.SchemaInfo; -import org.apache.carbondata.processing.constants.DataProcessorConstants; -import org.apache.carbondata.processing.csvreaderstep.CsvInputMeta; -import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus; -import org.apache.carbondata.processing.etl.DataLoadingException; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; - -import org.pentaho.di.core.KettleEnvironment; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.logging.LogLevel; -import org.pentaho.di.core.logging.LoggingObjectInterface; -import org.pentaho.di.core.logging.LoggingRegistry; -import org.pentaho.di.core.xml.XMLHandlerCache; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.steps.getfilenames.GetFileNamesMeta; -import org.pentaho.di.trans.steps.hadoopfileinput.HadoopFileInputMeta; -import org.pentaho.di.trans.steps.textfileinput.TextFileInputField; - -public class DataGraphExecuter { - /** - * Comment for LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(DataGraphExecuter.class.getName()); - /** - * graph transformation object - */ - private Trans trans; - /** - * - */ - private IDataProcessStatus model; - - public DataGraphExecuter(IDataProcessStatus model) { - this.model = model; - } - - /** - * This Method checks whether csv file provided and the column name in schema are same - * or not - * - * @param columnNames - * @param csvFilePath - * @return true if same, false otherwise. - */ - private boolean checkCSVAndRequestedTableColumns(String[] columnNames, String csvFilePath, - String delimiter) throws IOException { - return GraphExecutionUtil.checkCSVAndRequestedTableColumns(csvFilePath, columnNames, delimiter); - } - - /** - * This method returns the Columns names from the schema. - * - * @param tableName - * @return column names array. - */ - private String[] getColumnNames(String tableName, CarbonDataLoadSchema schema) { - Set columnNames = CarbonDataProcessorUtil.getSchemaColumnNames(schema, tableName); - return columnNames.toArray(new String[columnNames.size()]); - } - - private void validateCSV(String tableName, CarbonFile f, CarbonDataLoadSchema schema, - String delimiter) throws DataLoadingException, IOException { - - String[] columnNames = getColumnNames(tableName, schema); - - if (!checkCSVAndRequestedTableColumns(columnNames, f.getAbsolutePath(), delimiter)) { - LOGGER.error( - "CSV File provided is not proper. Column names in schema and csv header are not same. " - + "CSVFile Name : " - + f.getName()); - throw new DataLoadingException(DataProcessorConstants.CSV_VALIDATION_ERRROR_CODE, - "CSV File provided is not proper. Column names in schema and csv header are not same. " - + "CSVFile Name : " - + f.getName()); - } - } - - public void executeGraph(String graphFilePath, SchemaInfo schemaInfo, CarbonDataLoadSchema schema) - throws DataLoadingException { - - //This Method will validate the both fact and dimension csv files. - if (!schemaInfo.isAutoAggregateRequest() && model.getRddIteratorKey() == null) { - validateCSVFiles(schema); - } - execute(graphFilePath, schemaInfo); - } - - /** - * executeGraph which generate the kettle graph - * - * @throws DataLoadingException - */ - - private void execute(String graphFilePath, SchemaInfo schemaInfo) - throws DataLoadingException { - - //This Method will validate the both fact and dimension csv files. - - initKettleEnv(); - TransMeta transMeta = null; - try { - transMeta = new TransMeta(graphFilePath); - transMeta.setFilename(graphFilePath); - trans = new Trans(transMeta); - if (!schemaInfo.isAutoAggregateRequest()) { - // Register HDFS as a file system type with VFS to make HadoopFileInputMeta work - boolean hdfsReadMode = - model.getCsvFilePath() != null && model.getCsvFilePath().startsWith("hdfs:"); - trans.setVariable("modifiedDimNames", model.getDimTables()); - trans.setVariable("csvInputFilePath", model.getCsvFilePath()); - trans.setVariable(CarbonCommonConstants.BAD_RECORD_KEY, null); - if (hdfsReadMode) { - trans.addParameterDefinition("vfs.hdfs.dfs.client.read.shortcircuit", "true", ""); - trans.addParameterDefinition("vfs.hdfs.dfs.domain.socket.path", - "/var/lib/hadoop-hdfs-new/dn_socket", ""); - trans.addParameterDefinition("vfs.hdfs.dfs.block.local-path-access.user", "hadoop,root", - ""); - trans.addParameterDefinition("vfs.hdfs.io.file.buffer.size", "5048576", ""); - } - List stepsMeta = trans.getTransMeta().getSteps(); - StringBuilder builder = new StringBuilder(); - StringBuilder measuresInCSVFile = new StringBuilder(); - processCsvInputMeta(stepsMeta, builder, measuresInCSVFile); - processGetFileNamesMeta(stepsMeta); - - processHadoopFileInputMeta(stepsMeta, builder, measuresInCSVFile); - } - setGraphLogLevel(); - trans.execute(null); - LOGGER.info("Graph execution is started " + graphFilePath); - trans.waitUntilFinished(); - LOGGER.info("Graph execution is finished."); - } catch (KettleException | IOException e) { - LOGGER.error(e, "Unable to start execution of graph " + e.getMessage()); - throw new DataLoadingException("Unable to start execution of graph ", e); - } - - //Don't change the logic of creating key - String key = model.getDatabaseName() + '/' + model.getTableName() + '_' + model.getTableName(); - - if (trans.getErrors() > 0) { - if (null != trans.getVariable(CarbonCommonConstants.BAD_RECORD_KEY)) { - LOGGER.error(trans.getVariable(CarbonCommonConstants.BAD_RECORD_KEY)); - throw new DataLoadingException( - "Data load failed due to bad record ," + trans - .getVariable(CarbonCommonConstants.BAD_RECORD_KEY)); - } - LOGGER.error("Graph Execution had errors"); - throw new DataLoadingException("Due to internal errors, please check logs for more details."); - } else if (null != BadRecordsLogger.hasBadRecord(key)) { - LOGGER.error("Data load is partially success"); - throw new DataLoadingException(DataProcessorConstants.BAD_REC_FOUND, - "Data load is partially success"); - } else { - LOGGER.info("Graph execution task is over with No error."); - } - LoggingRegistry instance = LoggingRegistry.getInstance(); - Map map = instance.getMap(); - if (null != map) { - for (Entry entry : map.entrySet()) { - instance.removeIncludingChildren(entry.getKey()); - } - } - - map = null; - XMLHandlerCache.getInstance().clear(); - trans.cleanup(); - trans.eraseParameters(); - trans.killAll(); - trans = null; - } - - /** - * @param stepsMeta - * @param builder - * @param measuresInCSVFile - * @throws DataLoadingException - */ - private void processHadoopFileInputMeta(List stepsMeta, StringBuilder builder, - StringBuilder measuresInCSVFile) throws DataLoadingException { - for (StepMeta step : stepsMeta) { - if (step.getStepMetaInterface() instanceof HadoopFileInputMeta) { - - HadoopFileInputMeta stepMetaInterface = (HadoopFileInputMeta) step.getStepMetaInterface(); - if (null != model.getCsvFilePath()) { - stepMetaInterface.setFilenameField("filename"); - stepMetaInterface.setFileName(new String[] { "${csvInputFilePath}" }); - stepMetaInterface.setDefault(); - stepMetaInterface.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET); - stepMetaInterface.setEnclosure("\""); - stepMetaInterface.setHeader(true); - stepMetaInterface.setSeparator(","); - stepMetaInterface.setAcceptingFilenames(true); - stepMetaInterface.setAcceptingStepName("getFileNames"); - stepMetaInterface.setFileFormat("mixed"); - stepMetaInterface.setAcceptingField("filename"); - - CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath()); - TextFileInputField[] inputFields = GraphExecutionUtil - .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ","); - stepMetaInterface.setInputFields(inputFields); - } else if (model.isDirectLoad()) { - String[] files = new String[model.getFilesToProcess().size()]; - int i = 0; - for (String file : model.getFilesToProcess()) { - files[i++] = file; - } - stepMetaInterface.setFileName(files); - stepMetaInterface.setFilenameField("filename"); - stepMetaInterface.setDefault(); - stepMetaInterface.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET); - stepMetaInterface.setEnclosure("\""); - stepMetaInterface.setHeader(true); - stepMetaInterface.setSeparator(","); - stepMetaInterface.setAcceptingFilenames(true); - stepMetaInterface.setAcceptingStepName("getFileNames"); - stepMetaInterface.setFileFormat("mixed"); - stepMetaInterface.setAcceptingField("filename"); - - if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) { - TextFileInputField[] inputParams = GraphExecutionUtil - .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ","); - ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputParams); - ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()) - .setEscapeCharacter(model.getEscapeCharacter()); - ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false); - - } else if (model.getFilesToProcess().size() > 0) { - CarbonFile csvFile = - GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0)); - TextFileInputField[] inputFields = GraphExecutionUtil - .getTextInputFiles(csvFile, builder, measuresInCSVFile, - model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields); - ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()) - .setEscapeCharacter(model.getEscapeCharacter()); - ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true); - } - } - - break; - } - } - } - - /** - * @param stepsMeta - * @throws IOException - */ - private void processGetFileNamesMeta(List stepsMeta) throws IOException { - for (StepMeta step : stepsMeta) { - if (step.getStepMetaInterface() instanceof GetFileNamesMeta) { - GetFileNamesMeta stepMetaInterface = (GetFileNamesMeta) step.getStepMetaInterface(); - if (null != model.getCsvFilePath()) { - boolean checkIsFolder = GraphExecutionUtil.checkIsFolder(model.getCsvFilePath()); - if (checkIsFolder) { - stepMetaInterface.setFileName(new String[] { model.getCsvFilePath() }); - stepMetaInterface.setFileMask(new String[] { ".*\\.csv$|.*\\.inprogress" }); - stepMetaInterface.setExcludeFileMask(new String[] { "1" }); - } else { - //If absolute file path is provided for the data load and stopped in between then csv - // file will be - // changed to inprogress, and when next time server start then we need to check the - // file name extension. - // can contain .csv.inprogress file. - - FileType fileType = FileFactory.getFileType(model.getCsvFilePath()); - - boolean exists = FileFactory.isFileExist(model.getCsvFilePath(), fileType); - - if (exists) { - stepMetaInterface.setFileName(new String[] { model.getCsvFilePath() }); - stepMetaInterface.setExcludeFileMask(new String[] { null }); - } else { - stepMetaInterface.setFileName(new String[] { - model.getCsvFilePath() + CarbonCommonConstants.FILE_INPROGRESS_STATUS }); - stepMetaInterface.setExcludeFileMask(new String[] { null }); - } - } - } else if (model.isDirectLoad()) { - String[] files = new String[model.getFilesToProcess().size()]; - int i = 0; - for (String file : model.getFilesToProcess()) { - files[i++] = file; - } - stepMetaInterface.setFileName(files); - } - break; - } - } - } - - /** - * @param stepsMeta - * @param builder - * @param measuresInCSVFile - * @throws DataLoadingException - */ - private void processCsvInputMeta(List stepsMeta, StringBuilder builder, - StringBuilder measuresInCSVFile) throws DataLoadingException { - for (StepMeta step : stepsMeta) { - if (step.getStepMetaInterface() instanceof CsvInputMeta) { - if (null != model.getCsvFilePath() && model.getRddIteratorKey() == null) { - CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath()); - TextFileInputField[] inputFields = GraphExecutionUtil - .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ","); - ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields); - } else if (model.isDirectLoad()) { - if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) { - TextFileInputField[] inputFields = GraphExecutionUtil - .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ","); - ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields); - ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()) - .setEscapeCharacter(model.getEscapeCharacter()); - ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false); - - } else if (model.getFilesToProcess().size() > 0) { - CarbonFile csvFileToRead = - GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0)); - TextFileInputField[] inputFields = GraphExecutionUtil - .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, - model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields); - ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter()); - ((CsvInputMeta) step.getStepMetaInterface()) - .setEscapeCharacter(model.getEscapeCharacter()); - ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true); - } - } - break; - } - } - } - - /** - * - */ - private void initKettleEnv() { - try { - KettleEnvironment.init(false); - LOGGER.info("Kettle environment initialized"); - } catch (KettleException ke) { - LOGGER.error("Unable to initialize Kettle Environment " + ke.getMessage()); - } - } - - - private void setGraphLogLevel() { - trans.setLogLevel(LogLevel.NOTHING); - } - - /** - * This method will validate the both fact as well as dimension csv files. - * - * @param schema - * @throws DataLoadingException - */ - private void validateCSVFiles(CarbonDataLoadSchema schema) throws DataLoadingException { - // Validate the Fact CSV Files. - String csvFilePath = model.getCsvFilePath(); - if (csvFilePath != null) { - FileType fileType = FileFactory.getFileType(csvFilePath); - try { - boolean exists = FileFactory.isFileExist(csvFilePath, fileType); - if (exists && FileFactory.getCarbonFile(csvFilePath, fileType).isDirectory()) { - CarbonFile fileDir = FileFactory.getCarbonFile(csvFilePath, fileType); - CarbonFile[] listFiles = fileDir.listFiles(new CarbonFileFilter() { - - @Override public boolean accept(CarbonFile pathname) { - if (pathname.getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || pathname - .getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION - + CarbonCommonConstants.FILE_INPROGRESS_STATUS)) { - return true; - } - return false; - } - }); - - for (CarbonFile f : listFiles) { - validateCSV(model.getTableName(), f, schema, ","); - } - } else { - - if (!(csvFilePath.endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || csvFilePath - .endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION - + CarbonCommonConstants.FILE_INPROGRESS_STATUS))) { - LOGGER.error("File provided is not proper, Only csv files are allowed." + csvFilePath); - throw new DataLoadingException( - "File provided is not proper, Only csv files are allowed." + csvFilePath); - } - - if (exists) { - validateCSV(model.getTableName(), - FileFactory.getCarbonFile(csvFilePath, fileType), schema, ","); - } else { - validateCSV(model.getTableName(), FileFactory - .getCarbonFile(csvFilePath + CarbonCommonConstants.FILE_INPROGRESS_STATUS, - fileType), schema, ","); - } - - } - - } catch (IOException e) { - LOGGER.error(e, - "Error while checking file exists" + csvFilePath); - } - } else if (model.isDirectLoad()) { - if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) { - if (!CarbonDataProcessorUtil - .isHeaderValid(model.getTableName(), model.getCsvHeader(), schema, ",")) { - LOGGER.error("CSV header provided in DDL is not proper." - + " Column names in schema and CSV header are not the same."); - throw new DataLoadingException(DataProcessorConstants.CSV_VALIDATION_ERRROR_CODE, - "CSV header provided in DDL is not proper. Column names in schema and CSV header are " - + "not the same."); - } - } else { - for (String file : model.getFilesToProcess()) { - try { - FileFactory.FileType fileType = FileFactory.getFileType(file); - if (FileFactory.isFileExist(file, fileType)) { - validateCSV(model.getTableName(), - FileFactory.getCarbonFile(file, fileType), schema, - model.getCsvDelimiter()); - } - } catch (IOException e) { - LOGGER.error(e, - "Error while checking file exists" + file); - } - } - } - } - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java deleted file mode 100644 index 5504ba84000..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java +++ /dev/null @@ -1,242 +0,0 @@ -/* - * 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.csvload; - -import java.io.BufferedReader; -import java.io.DataInputStream; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStreamReader; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.filesystem.CarbonFile; -import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.datastore.impl.FileFactory.FileType; -import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.processing.etl.DataLoadingException; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema.DimensionRelation; - -import org.pentaho.di.trans.steps.textfileinput.TextFileInputField; - -public final class GraphExecutionUtil { - /** - * Comment for LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(GraphExecutionUtil.class.getName()); - - private GraphExecutionUtil() { - - } - - /** - * getCsvFileToRead - * - * @param csvFilePath - * @return File - */ - public static CarbonFile getCsvFileToRead(String csvFilePath) { - CarbonFile csvFile = - FileFactory.getCarbonFile(csvFilePath, FileFactory.getFileType(csvFilePath)); - - CarbonFile[] listFiles = null; - if (csvFile.isDirectory()) { - listFiles = csvFile.listFiles(new CarbonFileFilter() { - @Override public boolean accept(CarbonFile pathname) { - if (!pathname.isDirectory()) { - if (pathname.getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || pathname - .getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION - + CarbonCommonConstants.FILE_INPROGRESS_STATUS)) { - return true; - } - } - - return false; - } - }); - } else { - listFiles = new CarbonFile[1]; - listFiles[0] = csvFile; - - } - - return listFiles[0]; - } - - /** - * @param measuresInCSVFile - * @throws DataLoadingException - */ - public static TextFileInputField[] getTextInputFiles(CarbonFile csvFile, StringBuilder builder, - StringBuilder measuresInCSVFile, String delimiter) throws DataLoadingException { - DataInputStream fileReader = null; - BufferedReader bufferedReader = null; - String readLine = null; - - FileType fileType = FileFactory.getFileType(csvFile.getAbsolutePath()); - - if (!csvFile.exists()) { - csvFile = FileFactory - .getCarbonFile(csvFile.getAbsolutePath() + CarbonCommonConstants.FILE_INPROGRESS_STATUS, - fileType); - } - - try { - fileReader = FileFactory.getDataInputStream(csvFile.getAbsolutePath(), fileType); - bufferedReader = - new BufferedReader(new InputStreamReader(fileReader, Charset.defaultCharset())); - readLine = bufferedReader.readLine(); - } catch (FileNotFoundException e) { - LOGGER.error(e, "CSV Input File not found " + e.getMessage()); - throw new DataLoadingException("CSV Input File not found ", e); - } catch (IOException e) { - LOGGER.error(e, "Not able to read CSV input File " + e.getMessage()); - throw new DataLoadingException("Not able to read CSV input File ", e); - } finally { - CarbonUtil.closeStreams(fileReader, bufferedReader); - } - - if (null != readLine) { - delimiter = CarbonUtil.delimiterConverter(delimiter); - String[] columnNames = readLine.split(delimiter); - TextFileInputField[] textFileInputFields = new TextFileInputField[columnNames.length]; - - int i = 0; - String tmpCol; - for (String column : columnNames) { - tmpCol = column.replaceAll("\"", ""); - builder.append(tmpCol); - builder.append(";"); - textFileInputFields[i] = new TextFileInputField(); - textFileInputFields[i].setName(tmpCol.trim()); - textFileInputFields[i].setType(2); - measuresInCSVFile.append(tmpCol); - measuresInCSVFile.append(";"); - i++; - } - - return textFileInputFields; - } - - return null; - } - - /** - * @param measuresInCSVFile - * @throws DataLoadingException - */ - public static TextFileInputField[] getTextInputFiles(String header, StringBuilder builder, - StringBuilder measuresInCSVFile, String delimiter) throws DataLoadingException { - - String[] columnNames = header.split(delimiter); - TextFileInputField[] textFileInputFields = new TextFileInputField[columnNames.length]; - - int i = 0; - String tmpCol; - for (String columnName : columnNames) { - tmpCol = columnName.replaceAll("\"", ""); - builder.append(tmpCol); - builder.append(";"); - textFileInputFields[i] = new TextFileInputField(); - textFileInputFields[i].setName(tmpCol.trim()); - textFileInputFields[i].setType(2); - measuresInCSVFile.append(tmpCol); - measuresInCSVFile.append(";"); - i++; - } - - return textFileInputFields; - - } - - public static boolean checkIsFolder(String csvFilePath) { - try { - if (FileFactory.isFileExist(csvFilePath, FileFactory.getFileType(csvFilePath), false)) { - CarbonFile carbonFile = - FileFactory.getCarbonFile(csvFilePath, FileFactory.getFileType(csvFilePath)); - return carbonFile.isDirectory(); - } - } catch (IOException e) { - LOGGER.error(e, - "Not able check path exists or not " + e.getMessage() + "path: " + csvFilePath); - } - - return false; - } - - /** - * @param csvFilePath - * @param columnNames - * @return - */ - public static boolean checkCSVAndRequestedTableColumns(String csvFilePath, String[] columnNames, - String delimiter) throws IOException { - - String readLine = CarbonUtil.readHeader(csvFilePath); - - if (null != readLine) { - delimiter = CarbonUtil.delimiterConverter(delimiter); - String[] columnFromCSV = readLine.toLowerCase().split(delimiter); - - List csvColumnsList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - - for (String column : columnFromCSV) { - csvColumnsList.add(column.replaceAll("\"", "").trim()); - } - - int count = 0; - - for (String columns : columnNames) { - if (csvColumnsList.contains(columns.toLowerCase())) { - count++; - } - } - if (0 == count) { - LOGGER.error("There is No proper CSV file header found." + - " Either the ddl or the CSV file should provide CSV file header. "); - } - return (count == columnNames.length); - } - - return false; - } - - public static Set getDimensionColumnNames(String dimTableName, - CarbonDataLoadSchema schema) { - Set columnNames = new HashSet(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - for (DimensionRelation dimRel : schema.getDimensionRelationList()) { - if (dimRel.getTableName().equals(dimTableName)) { - for (String field : dimRel.getColumns()) { - columnNames.add(field); - } - break; - } - } - return columnNames; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java deleted file mode 100644 index 94bef1eca20..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.io.DataInputStream; -import java.io.IOException; -import java.io.InputStream; - -/** - * Custom reader class to read the data from file it will take care of reading - * till the limit assigned to this class - */ -public class BoundedDataStream extends InputStream { - - /** - * byte value of the new line character - */ - private static final byte END_OF_LINE_BYTE_VALUE = '\n'; - - /** - * number of extra character to read - */ - private static final int NUMBER_OF_EXTRA_CHARACTER_TO_READ = 100; - - /** - * number of bytes remaining - */ - private long remaining; - /** - * to check whether end of line is found - */ - private boolean endOfLineFound = false; - - private DataInputStream in; - - public BoundedDataStream(DataInputStream in, long limit) { - this.in = in; - this.remaining = limit; - } - - /** - * Below method will be used to read the data from file - * - * @throws IOException - * problem while reading - */ - @Override - public int read() throws IOException { - if (this.remaining == 0) { - return -1; - } else { - int var1 = this.in.read(); - if (var1 >= 0) { - --this.remaining; - } - - return var1; - } - } - - /** - * Below method will be used to read the data from file. If limit reaches in - * that case it will read until new line character is reached - * - * @param buffer - * buffer in which data will be read - * @param offset - * from position to buffer will be filled - * @param length - * number of character to be read - * @throws IOException - * problem while reading - */ - @Override - public int read(byte[] buffer, int offset, int length) throws IOException { - if (this.remaining == 0) { - return -1; - } else { - if (this.remaining < length) { - length = (int) this.remaining; - } - - length = this.in.read(buffer, offset, length); - if (length >= 0) { - this.remaining -= length; - if (this.remaining == 0 && !endOfLineFound) { - endOfLineFound = true; - this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ; - } else if (endOfLineFound) { - int end = offset + length; - for (int i = offset; i < end; i++) { - if (buffer[i] == END_OF_LINE_BYTE_VALUE) { - this.remaining = 0; - return (i - offset) + 1; - } - } - this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ; - } - } - return length; - } - } - - @Override - public void close() throws IOException { - if (in != null) { - in.close(); - } - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java deleted file mode 100644 index 6b65da02d70..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java +++ /dev/null @@ -1,431 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import org.apache.carbondata.common.CarbonIterator; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.common.logging.impl.StandardLogService; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory; -import org.apache.carbondata.processing.graphgenerator.GraphGenerator; - -import org.apache.commons.lang3.StringUtils; -import org.pentaho.di.core.Const; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.row.RowMeta; -import org.pentaho.di.core.row.ValueMetaInterface; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStep; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.pentaho.di.trans.steps.textfileinput.EncodingType; - -/** - * Read a simple CSV file - * Just output Strings found in the file... - */ -public class CsvInput extends BaseStep implements StepInterface { - private static final Class PKG = CsvInput.class; - // for i18n purposes, needed by Translator2!! $NON-NLS-1$ - private static final LogService LOGGER = - LogServiceFactory.getLogService(CsvInput.class.getName()); - /** - * NUM_CORES_DEFAULT_VAL - */ - private static final int NUM_CORES_DEFAULT_VAL = 2; - /** - * ReentrantLock getFileBlockLock - */ - private final Object getBlockListLock = new Object(); - /** - * ReentrantLock putRowLock - */ - private final Object putRowLock = new Object(); - private CsvInputMeta meta; - private CsvInputData data; - /** - * resultArray - */ - private Future[] resultArray; - private List> threadBlockList = new ArrayList<>(); - - private ExecutorService exec; - - /** - * If rddIteratorKey is not null, read data from RDD - */ - private String rddIteratorKey = null; - - private CarbonIterator> rddIterator; - - public CsvInput(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - super(stepMeta, stepDataInterface, copyNr, transMeta, trans); - LOGGER.info("** Using csv file **"); - } - - public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException { - meta = (CsvInputMeta) smi; - data = (CsvInputData) sdi; - - if (first) { - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValuesTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordCsvInputStepTime( - meta.getPartitionID(), System.currentTimeMillis()); - first = false; - data.outputRowMeta = new RowMeta(); - meta.getFields(data.outputRowMeta, getStepname(), null, null, this); - - // We only run in parallel if we have at least one file to process - // AND if we have more than one step copy running... - // - data.parallel = meta.isRunningInParallel() && data.totalNumberOfSteps > 1; - - // The conversion logic for when the lazy conversion is turned of is simple: - // Pretend it's a lazy conversion object anyway and get the native type during - // conversion. - // - data.convertRowMeta = data.outputRowMeta.clone(); - - for (ValueMetaInterface valueMeta : data.convertRowMeta.getValueMetaList()) { - valueMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING); - } - - // Calculate the indexes for the filename and row number fields - // - data.filenameFieldIndex = -1; - if (!Const.isEmpty(meta.getFilenameField()) && meta.isIncludingFilename()) { - data.filenameFieldIndex = meta.getInputFields().length; - } - - data.rownumFieldIndex = -1; - if (!Const.isEmpty(meta.getRowNumField())) { - data.rownumFieldIndex = meta.getInputFields().length; - if (data.filenameFieldIndex >= 0) { - data.rownumFieldIndex++; - } - } - rddIteratorKey = StringUtils.isEmpty(meta.getRddIteratorKey()) ? null : meta - .getRddIteratorKey(); - } - - // start multi-thread to process - int numberOfNodes; - try { - numberOfNodes = Integer.parseInt(CarbonProperties.getInstance() - .getProperty(CarbonCommonConstants.NUM_CORES_LOADING, - CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)); - } catch (NumberFormatException exc) { - numberOfNodes = NUM_CORES_DEFAULT_VAL; - } - if (rddIteratorKey == null) { - BlockDetails[] blocksInfo = GraphGenerator.blockInfo.get(meta.getBlocksID()); - if (blocksInfo.length == 0) { - //if isDirectLoad = true, and partition number > file num - //then blocksInfo will get empty in some partition processing, so just return - setOutputDone(); - return false; - } - - if (numberOfNodes > blocksInfo.length) { - numberOfNodes = blocksInfo.length; - } - - //new the empty lists - for (int pos = 0; pos < numberOfNodes; pos++) { - threadBlockList.add(new ArrayList()); - } - - //block balance to every thread - for (int pos = 0; pos < blocksInfo.length; ) { - for (int threadNum = 0; threadNum < numberOfNodes; threadNum++) { - if (pos < blocksInfo.length) { - threadBlockList.get(threadNum).add(blocksInfo[pos++]); - } - } - } - LOGGER.info("*****************Started all csv reading***********"); - startProcess(numberOfNodes); - LOGGER.info("*****************Completed all csv reading***********"); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordCsvInputStepTime( - meta.getPartitionID(), System.currentTimeMillis()); - } else if (rddIteratorKey.startsWith(CarbonCommonConstants.RDDUTIL_UPDATE_KEY)) { - scanRddIteratorForUpdate(); - } - else { - scanRddIterator(numberOfNodes); - } - setOutputDone(); - return false; - } - - class RddScanCallable implements Callable { - @Override public Void call() throws Exception { - StandardLogService - .setThreadName(("PROCESS_DataFrame_PARTITIONS"), Thread.currentThread().getName()); - try { - String[] values = null; - boolean hasNext = true; - CarbonIterator iter; - boolean isInitialized = false; - while (hasNext) { - // Inovke getRddIterator to get a RDD[Row] iterator of a partition. - // The RDD comes from the sub-query DataFrame in InsertInto statement. - iter = getRddIterator(isInitialized); - isInitialized = true; - if (iter == null) { - hasNext = false; - } else { - while (iter.hasNext()) { - values = iter.next(); - synchronized (putRowLock) { - putRow(data.outputRowMeta, values); - } - } - } - } - } catch (Exception e) { - LOGGER.error(e, "Scan rdd during data load is terminated due to error."); - throw e; - } - return null; - } - } - - private synchronized CarbonIterator getRddIterator(boolean isInitialized) { - if (!isInitialized) { - rddIterator.initialize(); - } - if (rddIterator.hasNext()) { - return rddIterator.next(); - } - return null; - } - - private void scanRddIterator(int numberOfNodes) throws RuntimeException { - rddIterator = RddInputUtils.getAndRemove(rddIteratorKey); - if (rddIterator != null) { - exec = Executors.newFixedThreadPool(numberOfNodes); - List> results = new ArrayList>(numberOfNodes); - RddScanCallable[] calls = new RddScanCallable[numberOfNodes]; - for (int i = 0; i < numberOfNodes; i++) { - calls[i] = new RddScanCallable(); - results.add(exec.submit(calls[i])); - } - try { - for (Future futrue : results) { - futrue.get(); - } - } catch (InterruptedException | ExecutionException e) { - LOGGER.error(e, "Thread InterruptedException"); - throw new RuntimeException("Thread InterruptedException", e); - } finally { - exec.shutdownNow(); - } - } - } - - private void scanRddIteratorForUpdate() throws RuntimeException { - Iterator iterator = RddInpututilsForUpdate.getAndRemove(rddIteratorKey); - if (iterator != null) { - try { - while (iterator.hasNext()) { - putRow(data.outputRowMeta, iterator.next()); - } - } catch (KettleException e) { - throw new RuntimeException(e); - } catch (Exception e) { - LOGGER.error(e, "Scan rdd during data load is terminated due to error."); - throw e; - } - } - } - - private void startProcess(final int numberOfNodes) throws RuntimeException { - exec = Executors.newFixedThreadPool(numberOfNodes); - - Callable callable = new Callable() { - @Override public Void call() throws RuntimeException { - StandardLogService.setThreadName(("PROCESS_BLOCKS"), Thread.currentThread().getName()); - try { - LOGGER.info("*****************started csv reading by thread***********"); - doProcessUnivocity(); - LOGGER.info("*****************Completed csv reading by thread***********"); - } catch (Throwable e) { - LOGGER.error(e, "Thread is terminated due to error"); - throw new RuntimeException("Thread is terminated due to error : " + e.getMessage()); - } - return null; - } - }; - List> results = new ArrayList>(10); - for (int i = 0; i < numberOfNodes; i++) { - results.add(exec.submit(callable)); - } - - resultArray = results.toArray(new Future[results.size()]); - try { - for (int j = 0; j < resultArray.length; j++) { - resultArray[j].get(); - } - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Thread InterruptedException", e); - } - finally { - exec.shutdownNow(); - } - } - - private void doProcessUnivocity() { - List blocksListForProcess = null; - synchronized (getBlockListLock) { - //get the blocksList for this thread - blocksListForProcess = threadBlockList.get(threadBlockList.size() - 1); - threadBlockList.remove(threadBlockList.size() - 1); - } - long currentTimeMillis = System.currentTimeMillis(); - UnivocityCsvParser parser = new UnivocityCsvParser(getParserVo(blocksListForProcess)); - long numberOfRows = 0; - int numberOfColumns = meta.getInputFields().length; - try { - parser.initialize(); - while (parser.hasMoreRecords()) { - String[] next = parser.getNextRecord(); - if (next.length < numberOfColumns) { - String[] temp = new String[numberOfColumns]; - System.arraycopy(next, 0, temp, 0, next.length); - next = temp; - } - synchronized (putRowLock) { - putRow(data.outputRowMeta, next); - numberOfRows++; - } - } - } catch (IOException e) { - throw new RuntimeException(e); - } catch (KettleException e) { - throw new RuntimeException(e); - } - LOGGER.info("Total Number of records processed by this thread is: " + numberOfRows); - LOGGER.info("Time taken to processed " + numberOfRows + " Number of records: " + ( - System.currentTimeMillis() - currentTimeMillis)); - } - - private UnivocityCsvParserVo getParserVo(List blocksListForProcess) { - UnivocityCsvParserVo csvParserVo = new UnivocityCsvParserVo(); - csvParserVo.setBlockDetailsList(blocksListForProcess); - csvParserVo.setDelimiter(meta.getDelimiter()); - csvParserVo.setNumberOfColumns(meta.getInputFields().length); - csvParserVo.setEscapeCharacter(meta.getEscapeCharacter()); - csvParserVo.setHeaderPresent(meta.isHeaderPresent()); - csvParserVo.setQuoteCharacter(meta.getQuoteCharacter()); - csvParserVo.setCommentCharacter(meta.getCommentCharacter()); - String maxColumns = meta.getMaxColumns(); - if (null != maxColumns) { - csvParserVo.setMaxColumns(Integer.parseInt(maxColumns)); - } - return csvParserVo; - } - - @Override public void dispose(StepMetaInterface smi, StepDataInterface sdi) { - try { - // Clean the block info in map - if (GraphGenerator.blockInfo.get(meta.getBlocksID()) != null) { - GraphGenerator.blockInfo.remove(meta.getBlocksID()); - } - } catch (Exception e) { - logError("Error closing file channel", e); - } - - super.dispose(smi, sdi); - } - - public boolean init(StepMetaInterface smi, StepDataInterface sdi) { - meta = (CsvInputMeta) smi; - data = (CsvInputData) sdi; - - if (super.init(smi, sdi)) { - // If the step doesn't have any previous steps, we just get the filename. - // Otherwise, we'll grab the list of filenames later... - // - if (getTransMeta().findNrPrevSteps(getStepMeta()) == 0) { - String filename = environmentSubstitute(meta.getFilename()); - - if (Const.isEmpty(filename) && Const.isEmpty(meta.getRddIteratorKey())) { - logError(BaseMessages.getString(PKG, "CsvInput.MissingFilename.Message")); //$NON-NLS-1$ - return false; - } - } - - data.encodingType = EncodingType.guessEncodingType(meta.getEncoding()); - - // PDI-2489 - set the delimiter byte value to the code point of the - // character as represented in the input file's encoding - try { - data.delimiter = data.encodingType - .getBytes(environmentSubstitute(meta.getDelimiter()), meta.getEncoding()); - data.escapeCharacter = data.encodingType - .getBytes(environmentSubstitute(meta.getEscapeCharacter()), meta.getEncoding()); - if (Const.isEmpty(meta.getEnclosure())) { - data.enclosure = null; - } else { - data.enclosure = data.encodingType - .getBytes(environmentSubstitute(meta.getEnclosure()), meta.getEncoding()); - } - - } catch (UnsupportedEncodingException e) { - logError(BaseMessages.getString(PKG, "CsvInput.BadEncoding.Message"), e); //$NON-NLS-1$ - return false; - } - - // Handle parallel reading capabilities... - // - - if (meta.isRunningInParallel()) { - data.totalNumberOfSteps = getUniqueStepCountAcrossSlaves(); - - // We are not handling a single file, but possibly a list of files... - // As such, the fair thing to do is calculate the total size of the files - // Then read the required block. - // - - } - return true; - } - return false; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java deleted file mode 100644 index 6114eb90fa5..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.csvreaderstep; - -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.step.BaseStepData; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.steps.textfileinput.EncodingType; - -public class CsvInputData extends BaseStepData implements StepDataInterface { - public RowMetaInterface convertRowMeta; - public RowMetaInterface outputRowMeta; - - public byte[] delimiter; - public byte[] enclosure; - public byte[] escapeCharacter; - public int preferredBufferSize; - public int totalNumberOfSteps; - public boolean parallel; - public int filenameFieldIndex; - public int rownumFieldIndex; - /** - *
-   * if true then when double enclosure appears one will be considered as escape enclosure
-   * Ecample: 'abc''xyz' would be processed as abc'xyz
-   * 
- */ - public EncodingType encodingType; - public CsvInputData() { - super(); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java deleted file mode 100644 index d0ba5852cfe..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java +++ /dev/null @@ -1,971 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.commons.vfs.FileObject; -import org.pentaho.di.core.CheckResult; -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Const; -import org.pentaho.di.core.Counter; -import org.pentaho.di.core.KettleAttributeInterface; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.exception.KettleStepException; -import org.pentaho.di.core.exception.KettleXMLException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.row.ValueMeta; -import org.pentaho.di.core.row.ValueMetaInterface; -import org.pentaho.di.core.variables.VariableSpace; -import org.pentaho.di.core.vfs.KettleVFS; -import org.pentaho.di.core.xml.XMLHandler; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.repository.ObjectId; -import org.pentaho.di.repository.Repository; -import org.pentaho.di.resource.ResourceDefinition; -import org.pentaho.di.resource.ResourceEntry; -import org.pentaho.di.resource.ResourceEntry.ResourceType; -import org.pentaho.di.resource.ResourceNamingInterface; -import org.pentaho.di.resource.ResourceReference; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStepMeta; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInjectionMetaEntry; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInjectionInterface; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.pentaho.di.trans.steps.textfileinput.InputFileMetaInterface; -import org.pentaho.di.trans.steps.textfileinput.TextFileInputField; -import org.pentaho.di.trans.steps.textfileinput.TextFileInputMeta; -import org.w3c.dom.Node; - -public class CsvInputMeta extends BaseStepMeta - implements StepMetaInterface, InputFileMetaInterface, StepMetaInjectionInterface { - private static final Class PKG = CsvInput.class; - - private String filename; - - private String filenameField; - - private boolean includingFilename; - - private String rowNumField; - - private boolean headerPresent; - - private String delimiter; - private String enclosure; - private boolean escapeEnclosure; - - private String bufferSize; - - private boolean lazyConversionActive; - - private TextFileInputField[] inputFields; - - private boolean isaddresult; - - private boolean runningInParallel; - - private String encoding; - - private boolean newlinePossibleInFields; - - private String blocksID; - - private String partitionID; - - private String escapeCharacter; - - private String quoteCharacter; - - private String commentCharacter; - - private String rddIteratorKey; - - private String maxColumns; - - public CsvInputMeta() { - super(); // allocate BaseStepMeta - allocate(0); - } - - public void loadXML(Node stepnode, List databases, Map counters) - throws KettleXMLException { - readData(stepnode); - } - - public void setDefault() { - delimiter = ","; - enclosure = "\""; - headerPresent = true; - lazyConversionActive = true; - isaddresult = false; - bufferSize = "50000"; - blocksID = ""; - partitionID = ""; - escapeCharacter = "\\"; - quoteCharacter = "\""; - commentCharacter = "#"; - rddIteratorKey = ""; - } - - private void readData(Node stepnode) throws KettleXMLException { - try { - filename = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME")); - filenameField = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME_FIELD")); - rowNumField = XMLHandler.getTagValue(stepnode, getXmlCode("ROW_NUM_FIELD")); - includingFilename = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("INCLUDE_FILENAME"))); - delimiter = XMLHandler.getTagValue(stepnode, getXmlCode("DELIMITER")); - enclosure = XMLHandler.getTagValue(stepnode, getXmlCode("ENCLOSURE")); - escapeEnclosure = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("ESACAPE_ENCLOSURE"))); - bufferSize = XMLHandler.getTagValue(stepnode, getXmlCode("BUFFERSIZE")); - headerPresent = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("HEADER_PRESENT"))); - lazyConversionActive = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("LAZY_CONVERSION"))); - isaddresult = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("ADD_FILENAME_RESULT"))); - runningInParallel = - "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("PARALLEL"))); - String nlp = XMLHandler.getTagValue(stepnode, getXmlCode("NEWLINE_POSSIBLE")); - if (Const.isEmpty(nlp)) { - if (runningInParallel) { - newlinePossibleInFields = false; - } else { - newlinePossibleInFields = true; - } - } else { - newlinePossibleInFields = "Y".equalsIgnoreCase(nlp); - } - encoding = XMLHandler.getTagValue(stepnode, getXmlCode("ENCODING")); - blocksID = XMLHandler.getTagValue(stepnode, "blocksID"); - partitionID = XMLHandler.getTagValue(stepnode, "partitionID"); - escapeCharacter = XMLHandler.getTagValue(stepnode, "escapeCharacter"); - quoteCharacter = XMLHandler.getTagValue(stepnode, "quoteCharacter"); - commentCharacter = XMLHandler.getTagValue(stepnode, "commentCharacter"); - rddIteratorKey = XMLHandler.getTagValue(stepnode, "rddIteratorKey"); - maxColumns = XMLHandler.getTagValue(stepnode, "maxColumns"); - Node fields = XMLHandler.getSubNode(stepnode, getXmlCode("FIELDS")); - int nrfields = XMLHandler.countNodes(fields, getXmlCode("FIELD")); - - allocate(nrfields); - - for (int i = 0; i < nrfields; i++) { - inputFields[i] = new TextFileInputField(); - - Node fnode = XMLHandler.getSubNodeByNr(fields, getXmlCode("FIELD"), i); - - inputFields[i].setName(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_NAME"))); - inputFields[i] - .setType(ValueMeta.getType(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_TYPE")))); - inputFields[i].setFormat(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_FORMAT"))); - inputFields[i] - .setCurrencySymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_CURRENCY"))); - inputFields[i].setDecimalSymbol(XMLHandler.getTagValue(fnode, - getXmlCode("FIELD_DECIMAL"))); - inputFields[i].setGroupSymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_GROUP"))); - inputFields[i] - .setLength(Const.toInt(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_LENGTH")), -1)); - inputFields[i].setPrecision( - Const.toInt(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_PRECISION")), -1)); - inputFields[i].setTrimType(ValueMeta - .getTrimTypeByCode(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_TRIM_TYPE")))); - } - } catch (Exception e) { - throw new KettleXMLException("Unable to load step info from XML", e); - } - } - - public void allocate(int nrFields) { - inputFields = new TextFileInputField[nrFields]; - } - - public String getXML() { - StringBuilder retval = new StringBuilder(500); - - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("FILENAME"), filename)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FILENAME_FIELD"), filenameField)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("ROW_NUM_FIELD"), rowNumField)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("INCLUDE_FILENAME"), includingFilename)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("DELIMITER"), delimiter)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("ENCLOSURE"), enclosure)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("ESACAPE_ENCLOSURE"), escapeEnclosure)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("HEADER_PRESENT"), headerPresent)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("BUFFERSIZE"), bufferSize)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("LAZY_CONVERSION"), lazyConversionActive)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("ADD_FILENAME_RESULT"), isaddresult)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("PARALLEL"), runningInParallel)); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("NEWLINE_POSSIBLE"), newlinePossibleInFields)); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("ENCODING"), encoding)); - retval.append(" ").append(XMLHandler.addTagValue("blocksID", blocksID)); - retval.append(" ").append(XMLHandler.addTagValue("partitionID", partitionID)); - retval.append(" ").append(XMLHandler.addTagValue("escapeCharacter", escapeCharacter)); - retval.append(" ").append(XMLHandler.addTagValue("quoteCharacter", quoteCharacter)); - retval.append(" ").append(XMLHandler.addTagValue("commentCharacter", commentCharacter)); - retval.append(" ").append(XMLHandler.addTagValue("rddIteratorKey", rddIteratorKey)); - retval.append(" ").append(XMLHandler.addTagValue("maxColumns", maxColumns)); - retval.append(" ").append(XMLHandler.openTag(getXmlCode("FIELDS"))).append(Const.CR); - for (int i = 0; i < inputFields.length; i++) { - TextFileInputField field = inputFields[i]; - - retval.append(" ").append(XMLHandler.openTag(getXmlCode("FIELD"))).append(Const.CR); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_NAME"), field.getName())); - retval.append(" ").append( - XMLHandler.addTagValue(getXmlCode("FIELD_TYPE"), ValueMeta.getTypeDesc(field.getType()))); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_FORMAT"), field.getFormat())); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_CURRENCY"), field.getCurrencySymbol())); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_DECIMAL"), field.getDecimalSymbol())); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_GROUP"), field.getGroupSymbol())); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_LENGTH"), field.getLength())); - retval.append(" ") - .append(XMLHandler.addTagValue(getXmlCode("FIELD_PRECISION"), field.getPrecision())); - retval.append(" ").append(XMLHandler.addTagValue(getXmlCode("FIELD_TRIM_TYPE"), - ValueMeta.getTrimTypeCode(field.getTrimType()))); - retval.append(" ").append(XMLHandler.closeTag(getXmlCode("FIELD"))).append(Const.CR); - } - retval.append(" ").append(XMLHandler.closeTag(getXmlCode("FIELDS"))).append(Const.CR); - - return retval.toString(); - } - - public void readRep(Repository rep, ObjectId idStep, List databases, - Map counters) throws KettleException { - try { - filename = rep.getStepAttributeString(idStep, getRepCode("FILENAME")); - filenameField = rep.getStepAttributeString(idStep, getRepCode("FILENAME_FIELD")); - rowNumField = rep.getStepAttributeString(idStep, getRepCode("ROW_NUM_FIELD")); - includingFilename = rep.getStepAttributeBoolean(idStep, getRepCode("INCLUDE_FILENAME")); - delimiter = rep.getStepAttributeString(idStep, getRepCode("DELIMITER")); - enclosure = rep.getStepAttributeString(idStep, getRepCode("ENCLOSURE")); - escapeEnclosure = rep.getStepAttributeBoolean(idStep, getRepCode("ESACAPE_ENCLOSURE")); - headerPresent = rep.getStepAttributeBoolean(idStep, getRepCode("HEADER_PRESENT")); - bufferSize = rep.getStepAttributeString(idStep, getRepCode("BUFFERSIZE")); - lazyConversionActive = rep.getStepAttributeBoolean(idStep, getRepCode("LAZY_CONVERSION")); - isaddresult = rep.getStepAttributeBoolean(idStep, getRepCode("ADD_FILENAME_RESULT")); - runningInParallel = rep.getStepAttributeBoolean(idStep, getRepCode("PARALLEL")); - newlinePossibleInFields = - rep.getStepAttributeBoolean(idStep, 0, getRepCode("NEWLINE_POSSIBLE"), - !runningInParallel); - encoding = rep.getStepAttributeString(idStep, getRepCode("ENCODING")); - blocksID = rep.getStepAttributeString(idStep, getRepCode("blocksID")); - partitionID = rep.getStepAttributeString(idStep, getRepCode("partitionID")); - escapeCharacter = rep.getStepAttributeString(idStep, getRepCode("escapeCharacter")); - quoteCharacter = rep.getStepAttributeString(idStep, getRepCode("quoteCharacter")); - commentCharacter = rep.getStepAttributeString(idStep, getRepCode("commentCharacter")); - rddIteratorKey = rep.getStepAttributeString(idStep, getRepCode("rddIteratorKey")); - maxColumns = rep.getStepAttributeString(idStep, getRepCode("maxColumns")); - int nrfields = rep.countNrStepAttributes(idStep, getRepCode("FIELD_NAME")); - - allocate(nrfields); - - for (int i = 0; i < nrfields; i++) { - inputFields[i] = new TextFileInputField(); - - inputFields[i].setName(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_NAME"))); - inputFields[i].setType( - ValueMeta.getType(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_TYPE")))); - inputFields[i].setFormat(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_FORMAT"))); - inputFields[i] - .setCurrencySymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_CURRENCY"))); - inputFields[i] - .setDecimalSymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_DECIMAL"))); - inputFields[i] - .setGroupSymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_GROUP"))); - inputFields[i] - .setLength((int) rep.getStepAttributeInteger(idStep, i, getRepCode("FIELD_LENGTH"))); - inputFields[i].setPrecision( - (int) rep.getStepAttributeInteger(idStep, i, getRepCode("FIELD_PRECISION"))); - inputFields[i].setTrimType(ValueMeta.getTrimTypeByCode( - rep.getStepAttributeString(idStep, i, getRepCode("FIELD_TRIM_TYPE")))); - } - } catch (Exception e) { - throw new KettleException("Unexpected error reading step information from the repository", e); - } - } - - public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep) - throws KettleException { - try { - rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME"), filename); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME_FIELD"), filenameField); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("ROW_NUM_FIELD"), rowNumField); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("INCLUDE_FILENAME"), - includingFilename); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("DELIMITER"), delimiter); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("ENCLOSURE"), enclosure); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("ESACAPE_ENCLOSURE"), - escapeEnclosure); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("BUFFERSIZE"), bufferSize); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("HEADER_PRESENT"), headerPresent); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("LAZY_CONVERSION"), - lazyConversionActive); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("ADD_FILENAME_RESULT"), - isaddresult); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("PARALLEL"), runningInParallel); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("NEWLINE_POSSIBLE"), - newlinePossibleInFields); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("ENCODING"), encoding); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("blocksID"), blocksID); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("partitionID"), partitionID); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("escapeCharacter"), - escapeCharacter); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("quoteCharacter"), - quoteCharacter); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("commentCharacter"), - commentCharacter); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("rddIteratorKey"), - rddIteratorKey); - rep.saveStepAttribute(idTransformation, idStep, getRepCode("maxColumns"), - maxColumns); - for (int i = 0; i < inputFields.length; i++) { - TextFileInputField field = inputFields[i]; - - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_NAME"), - field.getName()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_TYPE"), - ValueMeta.getTypeDesc(field.getType())); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_FORMAT"), - field.getFormat()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_CURRENCY"), - field.getCurrencySymbol()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_DECIMAL"), - field.getDecimalSymbol()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_GROUP"), - field.getGroupSymbol()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_LENGTH"), - field.getLength()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_PRECISION"), - field.getPrecision()); - rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_TRIM_TYPE"), - ValueMeta.getTrimTypeCode(field.getTrimType())); - } - } catch (Exception e) { - throw new KettleException( - "Unable to save step information to the repository for id_step=" + idStep, e); - } - } - - public void getFields(RowMetaInterface rowMeta, String origin, RowMetaInterface[] info, - StepMeta nextStep, VariableSpace space) throws KettleStepException { - rowMeta.clear(); // Start with a clean slate, eats the input - - for (int i = 0; i < inputFields.length; i++) { - TextFileInputField field = inputFields[i]; - - ValueMetaInterface valueMeta = new ValueMeta(field.getName(), field.getType()); - valueMeta.setConversionMask(field.getFormat()); - valueMeta.setLength(field.getLength()); - valueMeta.setPrecision(field.getPrecision()); - valueMeta.setConversionMask(field.getFormat()); - valueMeta.setDecimalSymbol(field.getDecimalSymbol()); - valueMeta.setGroupingSymbol(field.getGroupSymbol()); - valueMeta.setCurrencySymbol(field.getCurrencySymbol()); - valueMeta.setTrimType(field.getTrimType()); - if (lazyConversionActive) { - valueMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING); - } - valueMeta.setStringEncoding(space.environmentSubstitute(encoding)); - - // In case we want to convert Strings... - // Using a copy of the valueMeta object means that the inner and outer representation - // format is the same. - // Preview will show the data the same way as we read it. - // This layout is then taken further down the road by the metadata through the transformation. - // - ValueMetaInterface storageMetadata = valueMeta.clone(); - storageMetadata.setType(ValueMetaInterface.TYPE_STRING); - storageMetadata.setStorageType(ValueMetaInterface.STORAGE_TYPE_NORMAL); - storageMetadata - .setLength(-1, -1); // we don't really know the lengths of the strings read in advance. - valueMeta.setStorageMetadata(storageMetadata); - - valueMeta.setOrigin(origin); - - rowMeta.addValueMeta(valueMeta); - } - - if (!Const.isEmpty(filenameField) && includingFilename) { - ValueMetaInterface filenameMeta = - new ValueMeta(filenameField, ValueMetaInterface.TYPE_STRING); - filenameMeta.setOrigin(origin); - if (lazyConversionActive) { - filenameMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING); - filenameMeta - .setStorageMetadata(new ValueMeta(filenameField, ValueMetaInterface.TYPE_STRING)); - } - rowMeta.addValueMeta(filenameMeta); - } - - if (!Const.isEmpty(rowNumField)) { - ValueMetaInterface rowNumMeta = new ValueMeta(rowNumField, ValueMetaInterface.TYPE_INTEGER); - rowNumMeta.setLength(10); - rowNumMeta.setOrigin(origin); - rowMeta.addValueMeta(rowNumMeta); - } - - } - - public void check(List remarks, TransMeta transMeta, StepMeta stepinfo, - RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) { - CheckResult cr; - if (prev == null || prev.size() == 0) { - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK, - BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.NotReceivingFields"), - stepinfo); //$NON-NLS-1$ - remarks.add(cr); - } else { - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR, BaseMessages - .getString(PKG, "CsvInputMeta.CheckResult.StepRecevingData", prev.size() + ""), - stepinfo); //$NON-NLS-1$ //$NON-NLS-2$ - remarks.add(cr); - } - - // See if we have input streams leading to this step! - if (input.length > 0) { - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR, - BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.StepRecevingData2"), - stepinfo); //$NON-NLS-1$ - remarks.add(cr); - } else { - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK, - BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.NoInputReceivedFromOtherSteps"), - stepinfo); //$NON-NLS-1$ - remarks.add(cr); - } - } - - public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int cnr, - TransMeta tr, Trans trans) { - return new CsvInput(stepMeta, stepDataInterface, cnr, tr, trans); - } - - public StepDataInterface getStepData() { - return new CsvInputData(); - } - - /** - * @return the delimiter - */ - public String getDelimiter() { - return delimiter; - } - - /** - * @param delimiter the delimiter to set - */ - public void setDelimiter(String delimiter) { - this.delimiter = delimiter; - } - - /** - * @return the filename - */ - public String getFilename() { - return filename; - } - - /** - * @param filename the filename to set - */ - public void setFilename(String filename) { - this.filename = filename; - } - - /** - * @return the bufferSize - */ - public String getBufferSize() { - return bufferSize; - } - - /** - * @param bufferSize the bufferSize to set - */ - public void setBufferSize(String bufferSize) { - this.bufferSize = bufferSize; - } - - /** - * @return true if lazy conversion is turned on: conversions are delayed as long as possible, - * perhaps to never occur at all. - */ - public boolean isLazyConversionActive() { - return lazyConversionActive; - } - - /** - * @param lazyConversionActive true if lazy conversion is to be turned on: conversions are - * delayed as long as possible, perhaps to never occur at all. - */ - public void setLazyConversionActive(boolean lazyConversionActive) { - this.lazyConversionActive = lazyConversionActive; - } - - /** - * @return the headerPresent - */ - public boolean isHeaderPresent() { - return headerPresent; - } - - /** - * @param headerPresent the headerPresent to set - */ - public void setHeaderPresent(boolean headerPresent) { - this.headerPresent = headerPresent; - } - - /** - * @return the enclosure - */ - public String getEnclosure() { - return enclosure; - } - - /** - * @param enclosure the enclosure to set - */ - public void setEnclosure(String enclosure) { - this.enclosure = enclosure; - } - - /** - * @return the escapeEnclosure - */ - public boolean isEscapeEnclosure() { - return escapeEnclosure; - } - - /** - * @param escapeEnclosure the escapeEnclosure to set - */ - public void setEscapeEnclosure(boolean escapeEnclosure) { - this.escapeEnclosure = escapeEnclosure; - } - - @Override - public List getResourceDependencies(TransMeta transMeta, StepMeta stepInfo) { - List references = new ArrayList(5); - - ResourceReference reference = new ResourceReference(stepInfo); - references.add(reference); - if (!Const.isEmpty(filename)) { - // Add the filename to the references, including a reference to this - // step meta data. - // - reference.getEntries() - .add(new ResourceEntry(transMeta.environmentSubstitute(filename), ResourceType.FILE)); - } - return references; - } - - /** - * @return the inputFields - */ - public TextFileInputField[] getInputFields() { - return inputFields; - } - - /** - * @param inputFields the inputFields to set - */ - public void setInputFields(TextFileInputField[] inputFields) { - this.inputFields = inputFields; - } - - public int getFileFormatTypeNr() { - return TextFileInputMeta.FILE_FORMAT_MIXED; - } - - public String[] getFilePaths(VariableSpace space) { - return new String[] { space.environmentSubstitute(filename), }; - } - - public int getNrHeaderLines() { - return 1; - } - - public boolean hasHeader() { - return isHeaderPresent(); - } - - public String getErrorCountField() { - return null; - } - - public String getErrorFieldsField() { - return null; - } - - public String getErrorTextField() { - return null; - } - - public String getEscapeCharacter() { - return escapeCharacter; - } - - public void setEscapeCharacter(String escapeCharacter) { - this.escapeCharacter = escapeCharacter; - } - - public String getQuoteCharacter() { return quoteCharacter; } - - public void setQuoteCharacter(String quoteCharacter) { this.quoteCharacter = quoteCharacter; } - - public String getCommentCharacter() { return commentCharacter; } - - public void setCommentCharacter(String commentCharacter) { - this.commentCharacter = commentCharacter; - } - - public String getFileType() { - return "CSV"; - } - - public String getSeparator() { - return delimiter; - } - - public boolean includeFilename() { - return false; - } - - public boolean includeRowNumber() { - return false; - } - - public boolean isErrorIgnored() { - return false; - } - - public boolean isErrorLineSkipped() { - return false; - } - - /** - * @return the filenameField - */ - public String getFilenameField() { - return filenameField; - } - - /** - * @param filenameField the filenameField to set - */ - public void setFilenameField(String filenameField) { - this.filenameField = filenameField; - } - - /** - * @return the includingFilename - */ - public boolean isIncludingFilename() { - return includingFilename; - } - - /** - * @param includingFilename the includingFilename to set - */ - public void setIncludingFilename(boolean includingFilename) { - this.includingFilename = includingFilename; - } - - /** - * @return the rowNumField - */ - public String getRowNumField() { - return rowNumField; - } - - /** - * @param rowNumField the rowNumField to set - */ - public void setRowNumField(String rowNumField) { - this.rowNumField = rowNumField; - } - - /** - * @return Returns isaddresult. - */ - public boolean isAddResultFile() { - return isaddresult; - } - - /** - * @param isaddresult The isaddresult to set. - */ - public void setAddResultFile(boolean isaddresult) { - this.isaddresult = isaddresult; - } - - /** - * @return the runningInParallel - */ - public boolean isRunningInParallel() { - return runningInParallel; - } - - /** - * @param runningInParallel the runningInParallel to set - */ - public void setRunningInParallel(boolean runningInParallel) { - this.runningInParallel = runningInParallel; - } - - /** - * @return the encoding - */ - public String getEncoding() { - return encoding; - } - - /** - * @param encoding the encoding to set - */ - public void setEncoding(String encoding) { - this.encoding = encoding; - } - - public String getBlocksID() { - return blocksID; - } - - public void setBlocksID(String blocksID) { - this.blocksID = blocksID; - } - - /** - * Since the exported transformation that runs this will reside in a ZIP file, we can't - * reference files relatively. - * So what this does is turn the name of files into absolute paths OR it simply includes the - * resource in the ZIP file. - * For now, we'll simply turn it into an absolute path and pray that the file is on a shared - * drive or something like that. - * HANDLER: create options to configure this behavior - */ - public String exportResources(VariableSpace space, Map definitions, - ResourceNamingInterface resourceNamingInterface, Repository repository) - throws KettleException { - try { - // The object that we're modifying here is a copy of the original! - // So let's change the filename from relative to absolute by grabbing the file object... - // In case the name of the file comes from previous steps, forget about this! - // - if (Const.isEmpty(filenameField)) { - // From : ${Internal.Transformation.Filename.Directory}/../foo/bar.csv - // To : /home/matt/test/files/foo/bar.csv - // - FileObject fileObject = - KettleVFS.getFileObject(space.environmentSubstitute(filename), space); - - // If the file doesn't exist, forget about this effort too! - // - if (fileObject.exists()) { - // Convert to an absolute path... - // - filename = resourceNamingInterface.nameResource(fileObject, space, true); - - return filename; - } - } - return null; - } catch (Exception e) { - throw new KettleException(e); //$NON-NLS-1$ - } - } - - public boolean supportsErrorHandling() { - return true; - } - - public StepMetaInjectionInterface getStepMetaInjectionInterface() { - - return this; - } - - public void injectStepMetadataEntries(List metadata) { - for (StepInjectionMetaEntry entry : metadata) { - KettleAttributeInterface attr = findAttribute(entry.getKey()); - - // Set top level attributes... - // - String attributeKey = attr.getKey(); - if (entry.getValueType() != ValueMetaInterface.TYPE_NONE) { - if ("FILENAME".equals(attributeKey)) { - filename = (String) entry.getValue(); - } else if ("FILENAME_FIELD".equals(attributeKey)) { - filenameField = (String) entry.getValue(); - } else if ("ROW_NUM_FIELD".equals(attributeKey)) { - rowNumField = (String) entry.getValue(); - } else if ("HEADER_PRESENT".equals(attributeKey)) { - headerPresent = (Boolean) entry.getValue(); - } else if ("DELIMITER".equals(attributeKey)) { - delimiter = (String) entry.getValue(); - } else if ("ENCLOSURE".equals(attributeKey)) { - enclosure = (String) entry.getValue(); - } else if ("ESACAPE_ENCLOSURE".equals(attributeKey)) { - escapeEnclosure = (Boolean) entry.getValue(); - } else if ("BUFFERSIZE".equals(attributeKey)) { - bufferSize = (String) entry.getValue(); - } else if ("LAZY_CONVERSION".equals(attributeKey)) { - lazyConversionActive = (Boolean) entry.getValue(); - } else if ("PARALLEL".equals(attributeKey)) { - runningInParallel = (Boolean) entry.getValue(); - } else if ("NEWLINE_POSSIBLE".equals(attributeKey)) { - newlinePossibleInFields = (Boolean) entry.getValue(); - } else if ("ADD_FILENAME_RESULT".equals(attributeKey)) { - isaddresult = (Boolean) entry.getValue(); - } else if ("ENCODING".equals(attributeKey)) { - encoding = (String) entry.getValue(); - } else if ("blocksID".equals(attributeKey)) { - blocksID = (String) entry.getValue(); - } else if ("partitionID".equals(attributeKey)) { - partitionID = (String) entry.getValue(); - } else if ("escapeCharacter".equals(attributeKey)) { - escapeCharacter = (String) entry.getValue(); - } else if ("quoteCharacter".equals(attributeKey)) { - quoteCharacter = (String) entry.getValue(); - } else if ("commentCharacter".equals(attributeKey)) { - commentCharacter = (String) entry.getValue(); - } else if ("rddIteratorKey".equals(attributeKey)) { - rddIteratorKey = (String) entry.getValue(); - } else { - throw new RuntimeException( - "Unhandled metadata injection of attribute: " + attr.toString() + " - " + attr - .getDescription()); - } - } else { - if ("FIELDS".equals(attributeKey)) { - // This entry contains a list of lists... - // Each list contains a single CSV input field definition (one line in the dialog) - // - List inputFieldEntries = entry.getDetails(); - inputFields = new TextFileInputField[inputFieldEntries.size()]; - for (int row = 0; row < inputFieldEntries.size(); row++) { - StepInjectionMetaEntry inputFieldEntry = inputFieldEntries.get(row); - TextFileInputField inputField = new TextFileInputField(); - - List fieldAttributes = inputFieldEntry.getDetails(); - for (int i = 0; i < fieldAttributes.size(); i++) { - StepInjectionMetaEntry fieldAttribute = fieldAttributes.get(i); - KettleAttributeInterface fieldAttr = findAttribute(fieldAttribute.getKey()); - - String attributeValue = (String) fieldAttribute.getValue(); - String fieldAttributeKey = fieldAttr.getKey(); - if ("FIELD_NAME".equals(fieldAttributeKey)) { - inputField.setName(attributeValue); - } else if ("FIELD_TYPE".equals(fieldAttributeKey)) { - inputField.setType(ValueMeta.getType(attributeValue)); - } else if ("FIELD_FORMAT".equals(fieldAttributeKey)) { - inputField.setFormat(attributeValue); - } else if ("FIELD_LENGTH".equals(fieldAttributeKey)) { - inputField - .setLength(attributeValue == null ? -1 : Integer.parseInt(attributeValue)); - } else if ("FIELD_PRECISION".equals(fieldAttributeKey)) { - inputField - .setPrecision(attributeValue == null ? -1 : Integer.parseInt(attributeValue)); - } else if ("FIELD_CURRENCY".equals(fieldAttributeKey)) { - inputField.setCurrencySymbol(attributeValue); - } else if ("FIELD_DECIMAL".equals(fieldAttributeKey)) { - inputField.setDecimalSymbol(attributeValue); - } else if ("FIELD_GROUP".equals(fieldAttributeKey)) { - inputField.setGroupSymbol(attributeValue); - } else if ("FIELD_TRIM_TYPE".equals(fieldAttributeKey)) { - inputField.setTrimType(ValueMeta.getTrimTypeByCode(attributeValue)); - } else { - throw new RuntimeException( - "Unhandled metadata injection of attribute: " + fieldAttr.toString() + " - " - + fieldAttr.getDescription()); - } - } - - inputFields[row] = inputField; - } - } - } - } - } - - /** - * Describe the metadata attributes that can be injected into this step metadata object. - * - * @throws KettleException - */ - public List getStepInjectionMetadataEntries() throws KettleException { - return getStepInjectionMetadataEntries(PKG); - } - - /** - * @return the newlinePossibleInFields - */ - public boolean isNewlinePossibleInFields() { - return newlinePossibleInFields; - } - - /** - * @param newlinePossibleInFields the newlinePossibleInFields to set - */ - public void setNewlinePossibleInFields(boolean newlinePossibleInFields) { - this.newlinePossibleInFields = newlinePossibleInFields; - } - - public void setPartitionID(String partitionID) { - this.partitionID = partitionID; - } - - public String getPartitionID() { - return this.partitionID; - } - - public String getMaxColumns() { - return maxColumns; - } - - public void setMaxColumns(String maxColumns) { - this.maxColumns = maxColumns; - } - - public String getRddIteratorKey() { - return this.rddIteratorKey; - } - public void setRddIteratorKey(String rddIteratorKey) { - this.rddIteratorKey = rddIteratorKey; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java deleted file mode 100644 index 0f5add26fdb..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.carbondata.common.CarbonIterator; - -public class RddInputUtils { - private static Map>> iteratorMap = new - HashMap>>(); - - public static void put(String key, CarbonIterator> value) { - iteratorMap.put(key, value); - } - - public static CarbonIterator> getAndRemove(String key) { - CarbonIterator> iter = iteratorMap.get(key); - remove(key); - return iter; - } - - public static void remove(String key) { - iteratorMap.remove(key); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java deleted file mode 100644 index b0f26e5f5ce..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; - -public class RddInpututilsForUpdate { - private static Map> iteratorMap = new HashMap>(); - - public static void put(String key, Iterator value) { - iteratorMap.put(key, value); - } - - public static Iterator getAndRemove(String key) { - Iterator iter = iteratorMap.get(key); - remove(key); - return iter; - } - - public static void remove(String key) { - iteratorMap.remove(key); - } -} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java deleted file mode 100644 index f99fab4647a..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java +++ /dev/null @@ -1,224 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.io.BufferedReader; -import java.io.DataInputStream; -import java.io.IOException; -import java.io.InputStreamReader; -import java.io.Reader; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.datastore.impl.FileFactory.FileType; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonUtil; - -import com.univocity.parsers.csv.CsvParser; -import com.univocity.parsers.csv.CsvParserSettings; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.util.LineReader; - -/** - * Class which will be used to read the data from csv file and parse the record - */ -public class UnivocityCsvParser { - - private LogService LOGGER = LogServiceFactory.getLogService(this.getClass().getName()); - - /** - * Max number of columns that will be parsed for a row by univocity parsing - */ - private static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000; - /** - * Maximum allowed value for number of columns to be parsed in each row - */ - private static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000; - /** - * reader for csv - */ - private Reader inputStreamReader; - - /** - * buffer size of stream - */ - private int bufferSize; - - /** - * to keep track how many block has been processed - */ - private int blockCounter = -1; - - /** - * csv record parser which read and convert the record to csv format - */ - private CsvParser parser; - - /** - * row from csv - */ - private String[] row; - - /** - * holding all the properties required for parsing the records - */ - private UnivocityCsvParserVo csvParserVo; - - public UnivocityCsvParser(UnivocityCsvParserVo csvParserVo) { - this.csvParserVo = csvParserVo; - bufferSize = Integer.parseInt(CarbonProperties.getInstance() - .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE, - CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT)); - } - - /** - * Below method will be used to initialize the the parser - * - * @throws IOException - */ - public void initialize() throws IOException { - CsvParserSettings parserSettings = new CsvParserSettings(); - parserSettings.getFormat().setDelimiter(csvParserVo.getDelimiter().charAt(0)); - parserSettings.getFormat().setComment(csvParserVo.getCommentCharacter().charAt(0)); - parserSettings.setLineSeparatorDetectionEnabled(true); - parserSettings.setMaxColumns( - getMaxColumnsForParsing(csvParserVo.getNumberOfColumns(), csvParserVo.getMaxColumns())); - parserSettings.setNullValue(""); - parserSettings.setEmptyValue(""); - parserSettings.setIgnoreLeadingWhitespaces(false); - parserSettings.setIgnoreTrailingWhitespaces(false); - parserSettings.setSkipEmptyLines(false); - parserSettings.getFormat().setQuote( - null == csvParserVo.getQuoteCharacter() ? '\"' : csvParserVo.getQuoteCharacter().charAt(0)); - parserSettings.getFormat().setQuoteEscape(null == csvParserVo.getEscapeCharacter() ? - '\\' : - csvParserVo.getEscapeCharacter().charAt(0)); - blockCounter++; - initializeReader(); - if (csvParserVo.getBlockDetailsList().get(blockCounter).getBlockOffset() == 0) { - parserSettings.setHeaderExtractionEnabled(csvParserVo.isHeaderPresent()); - } - parser = new CsvParser(parserSettings); - parser.beginParsing(inputStreamReader); - } - - /** - * This method will decide the number of columns to be parsed for a row by univocity parser - * - * @param columnCountInSchema total number of columns in schema - * @return - */ - private int getMaxColumnsForParsing(int columnCountInSchema, int maxColumns) { - int maxNumberOfColumnsForParsing = DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING; - if (maxColumns > 0) { - if (columnCountInSchema >= maxColumns) { - // univocity parser needs one extra count from the number of columns - // specified during processing. eg. columnCount=12, then array size should be 13 - maxNumberOfColumnsForParsing = columnCountInSchema + 1; - } else if (maxColumns > THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) { - maxNumberOfColumnsForParsing = THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING; - LOGGER.info("MAXCOLUMNS option value configured is more than system allowed limit. " - + "Therefore threshold value for max column parsing will be considered: " - + THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING); - } else { - maxNumberOfColumnsForParsing = maxColumns; - } - } else if (columnCountInSchema >= DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) { - // univocity parser needs one extra count from the number of columns - // specified during processing. eg. columnCount=2200, then array size should be 2201 - maxNumberOfColumnsForParsing = columnCountInSchema + 1; - } - return maxNumberOfColumnsForParsing; - } - - /** - * Below method will be used to initialize the reader - * - * @throws IOException - */ - private void initializeReader() throws IOException { - // if already one input stream is open first we need to close and then - // open new stream - close(); - - String path = this.csvParserVo.getBlockDetailsList().get(blockCounter).getFilePath(); - FileType fileType = FileFactory.getFileType(path); - - if (path.endsWith(".gz") || - path.endsWith(".bz2")) { - DataInputStream dataInputStream = FileFactory.getDataInputStream(path, fileType, bufferSize); - inputStreamReader = new BufferedReader(new InputStreamReader(dataInputStream)); - } else { - long startOffset = this.csvParserVo.getBlockDetailsList().get(blockCounter).getBlockOffset(); - long blockLength = this.csvParserVo.getBlockDetailsList().get(blockCounter).getBlockLength(); - long endOffset = blockLength + startOffset; - - DataInputStream dataInputStream = - FileFactory.getDataInputStream(path, fileType, bufferSize, startOffset); - // if start offset is not 0 then reading then reading and ignoring the extra line - if (startOffset != 0) { - LineReader lineReader = new LineReader(dataInputStream, 1); - startOffset += lineReader.readLine(new Text(), 0); - } - inputStreamReader = new BufferedReader(new InputStreamReader( - new BoundedDataStream(dataInputStream, endOffset - startOffset))); - } - } - - /** - * Below method will be used to clear all the stream - */ - public void close() { - if (null != inputStreamReader) { - CarbonUtil.closeStreams(inputStreamReader); - } - - } - - /** - * Below method will be used to check whether any more records is present or - * not - * - * @return true if more records are present - * @throws IOException - */ - public boolean hasMoreRecords() throws IOException { - row = parser.parseNext(); - if (row == null && blockCounter + 1 >= this.csvParserVo.getBlockDetailsList().size()) { - close(); - return false; - } - if (row == null) { - initialize(); - row = parser.parseNext(); - } - return true; - } - - /** - * Below method will be used to get the new record - * - * @return next record - */ - public String[] getNextRecord() { - String[] returnValue = row; - row = null; - return returnValue; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java deleted file mode 100644 index cb6c4324fb6..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java +++ /dev/null @@ -1,220 +0,0 @@ -/* - * 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.csvreaderstep; - -import java.util.List; - -/** - * Vo class which will holds all the properties required to read and parse the - * csv file - */ -public class UnivocityCsvParserVo { - - /** - * delimiter of the records - */ - private String delimiter; - - /** - * file encoding - */ - private String encoding; - - /** - * is header present in the file - */ - private boolean headerPresent; - - /** - * enclosure - */ - private String enclosure; - - /** - * escape enclosure - */ - private boolean escapeEnclosure; - - /** - * number of columns - */ - private int numberOfColumns; - - /** - * block details list, which will have - * all the detail if the block - */ - private List blockDetailsList; - - /** - * escape character; - */ - private String escapeCharacter; - - /** - * quote character; - */ - private String quoteCharacter; - - /** - * comment character; - */ - private String commentCharacter; - - /** - * max number of columns configured by user to be parsed in a row - */ - private int maxColumns; - - /** - * @return the delimiter - */ - public String getDelimiter() { - return delimiter; - } - - /** - * @param delimiter the delimiter to set - */ - public void setDelimiter(String delimiter) { - this.delimiter = delimiter; - } - - /** - * @return the encoding - */ - public String getEncoding() { - return encoding; - } - - /** - * @param encoding the encoding to set - */ - public void setEncoding(String encoding) { - this.encoding = encoding; - } - - /** - * @return the headerPresent - */ - public boolean isHeaderPresent() { - return headerPresent; - } - - /** - * @param headerPresent the headerPresent to set - */ - public void setHeaderPresent(boolean headerPresent) { - this.headerPresent = headerPresent; - } - - /** - * @return the enclosure - */ - public String getEnclosure() { - return enclosure; - } - - /** - * @param enclosure the enclosure to set - */ - public void setEnclosure(String enclosure) { - this.enclosure = enclosure; - } - - /** - * @return the escapeEnclosure - */ - public boolean isEscapeEnclosure() { - return escapeEnclosure; - } - - /** - * @param escapeEnclosure the escapeEnclosure to set - */ - public void setEscapeEnclosure(boolean escapeEnclosure) { - this.escapeEnclosure = escapeEnclosure; - } - - /** - * @return the numberOfColumns - */ - public int getNumberOfColumns() { - return numberOfColumns; - } - - /** - * @param numberOfColumns the numberOfColumns to set - */ - public void setNumberOfColumns(int numberOfColumns) { - this.numberOfColumns = numberOfColumns; - } - - /** - * @return the blockDetailsList - */ - public List getBlockDetailsList() { - return blockDetailsList; - } - - /** - * @param blockDetailsList the blockDetailsList to set - */ - public void setBlockDetailsList(List blockDetailsList) { - this.blockDetailsList = blockDetailsList; - } - - /** - * @return the escapeCharacter - */ - public String getEscapeCharacter() { - return escapeCharacter; - } - - /** - * @param escapeCharacter the escapeCharacter to set - */ - public void setEscapeCharacter(String escapeCharacter) { - this.escapeCharacter = escapeCharacter; - } - - public String getQuoteCharacter() { return quoteCharacter; } - - public void setQuoteCharacter(String quoteCharacter) { this.quoteCharacter = quoteCharacter; } - - public String getCommentCharacter() { return commentCharacter; } - - public void setCommentCharacter(String commentCharacter) { - this.commentCharacter = commentCharacter; - } - - - /** - * @return - */ - public int getMaxColumns() { - return maxColumns; - } - - /** - * @param maxColumns - */ - public void setMaxColumns(int maxColumns) { - this.maxColumns = maxColumns; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml deleted file mode 100644 index 32b1452be7f..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml +++ /dev/null @@ -1,229 +0,0 @@ - - - - - - - filename - - CsvInputDialog.Filename.Label - - String - - - - filename_field - - CsvInputDialog.FilenameField.Label - - String - - - - include_filename - - CsvInputDialog.IncludeFilenameField.Label - - Boolean - - - - rownum_field - - CsvInputDialog.RowNumField.Label - - String - - - - header - - CsvInputDialog.HeaderPresent.Label - - Boolean - - - - separator - - CsvInputDialog.Delimiter.Label - - String - - - - enclosure - - CsvInputDialog.Enclosure.Label - - String - - - - escape_enclosure - - CsvInputDialog.EscapeEnclosure.Label - - Boolean - - - - buffer_size - - CsvInputDialog.BufferSize.Label - - String - - - - lazy_conversion - - CsvInputDialog.LazyConversion.Label - - Boolean - - - - parallel - - CsvInputDialog.RunningInParallel.Label - - Boolean - - - - newline_possible - - CsvInputDialog.NewlinePossible.Label - - Boolean - - - - add_filename_result - - CsvInputDialog.AddResult.Label - CsvInputDialog.AddResult.Tooltip - Boolean - - - - encoding - - CsvInputDialog.Encoding.Label - - String - - - - - - fields - - CsvInputDialog.Fields.Label - - ValueMetaInterface.TYPE_NONE - - - - - - field - - CsvInputDialog.Field.Label - - ValueMetaInterface.TYPE_NONE - FIELDS - - - - name - field_name - CsvInputDialog.NameColumn.Column - - String - FIELD - - - type - field_type - CsvInputDialog.TypeColumn.Column - - String - FIELD - - - format - field_format - CsvInputDialog.FormatColumn.Column - - String - FIELD - - - length - field_length - CsvInputDialog.LengthColumn.Column - - String - FIELD - - - precision - field_precision - CsvInputDialog.PrecisionColumn.Column - - String - FIELD - - - currency - field_currency - CsvInputDialog.CurrencyColumn.Column - - String - FIELD - - - mantissa - field_decimal - CsvInputDialog.DecimalColumn.Column - - String - FIELD - - - group - field_group - CsvInputDialog.GroupColumn.Column - - String - FIELD - - - trim_type - field_trim_type - CsvInputDialog.TrimTypeColumn.Column - - String - FIELD - - - \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java deleted file mode 100644 index eba739a92ba..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java +++ /dev/null @@ -1,301 +0,0 @@ -/* - * 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.dataprocessor; - -import java.io.Serializable; -import java.sql.Timestamp; -import java.util.List; - -public class DataProcessTaskStatus implements IDataProcessStatus, Serializable { - - private static final long serialVersionUID = 1L; - - /** - * DataLoader Status Identifier. - */ - private int dataloadstatusid; - - /** - * - */ - private Timestamp createdTime; - - /** - * Status Identifier. - */ - private String key; - - /** - * Status . - */ - private String status; - - /** - * description for the task - */ - private String desc; - - /** - * task type - */ - private int taskType; - - private String databaseName; - - private String tableName; - - private String newSchemaFilePath; - - private String oldSchemaFilePath; - - private String csvFilePath; - - private String dimTables; - - private boolean isDirectLoad; - private List filesToProcess; - private String csvHeader; - private String csvDelimiter; - /** - * Set if the call to restructre from path or by upload - */ - private boolean isFromPathApi; - - private String blocksID; - - private String escapeCharacter; - - private String quoteCharacter; - - private String commentCharacter; - - private String rddIteratorKey; - - private String dateFormat; - - public DataProcessTaskStatus(String databaseName, String tableName) { - this.databaseName = databaseName; - this.tableName = tableName; - this.desc = ""; - this.setNewSchemaFilePath(""); - this.setOldSchemaFilePath(""); - } - - public DataProcessTaskStatus() { - } - - public boolean isDirectLoad() { - return isDirectLoad; - } - - public void setDirectLoad(boolean isDirectLoad) { - this.isDirectLoad = isDirectLoad; - } - - public List getFilesToProcess() { - return filesToProcess; - } - - public void setFilesToProcess(List filesToProcess) { - this.filesToProcess = filesToProcess; - } - - public String getCsvHeader() { - return csvHeader; - } - - public void setCsvHeader(String csvHeader) { - this.csvHeader = csvHeader; - } - - public String getCsvDelimiter() { - return csvDelimiter; - } - - public void setCsvDelimiter(String csvDelimiter) { - this.csvDelimiter = csvDelimiter; - } - - /** - * @return the databaseName - */ - public String getDatabaseName() { - return databaseName; - } - - /** - * @param databaseName the databaseName to set - */ - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - /** - * @return the tableName - */ - public String getTableName() { - return tableName; - } - - /** - * @param tableName the tableName to set - */ - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getDesc() { - return desc; - } - - public void setDesc(String desc) { - this.desc = desc; - } - - @Override public String getKey() { - return key; - } - - public void setKey(String key) { - this.key = key; - } - - @Override public String getStatus() { - return status; - } - - public void setStatus(String status) { - this.status = status; - } - - public int getDataloadstatusid() { - return dataloadstatusid; - } - - public void setDataloadstatusid(int dataloadstatusid) { - this.dataloadstatusid = dataloadstatusid; - } - - public Timestamp getCreatedTime() { - return createdTime; - } - - public void setCreatedTime(Timestamp createdTime) { - this.createdTime = createdTime; - } - - public int getTaskType() { - return taskType; - } - - public void setTaskType(int taskType) { - this.taskType = taskType; - } - - public String getOldSchemaFilePath() { - return oldSchemaFilePath; - } - - public void setOldSchemaFilePath(String oldSchemaFilePath) { - this.oldSchemaFilePath = oldSchemaFilePath; - } - - public String getNewSchemaFilePath() { - return newSchemaFilePath; - } - - public void setNewSchemaFilePath(String newSchemaFilePath) { - this.newSchemaFilePath = newSchemaFilePath; - } - - public String getCsvFilePath() { - return csvFilePath; - } - - public void setCsvFilePath(String csvFilePath) { - this.csvFilePath = csvFilePath; - } - - public String getDimTables() { - return dimTables; - } - - public void setDimTables(String dimTables) { - this.dimTables = dimTables; - } - - public boolean isFromPathApi() { - return isFromPathApi; - } - - public void setFromPathApi(boolean isFromPathApi) { - this.isFromPathApi = isFromPathApi; - } - - /** - * to make a copy - */ - public IDataProcessStatus makeCopy() { - IDataProcessStatus copy = new DataProcessTaskStatus(); - copy.setTableName(this.tableName); - copy.setDataloadstatusid(this.dataloadstatusid); - copy.setDesc(this.desc); - copy.setKey(this.key); - copy.setDatabaseName(databaseName); - copy.setStatus(status); - return copy; - } - - public String getBlocksID() { - return blocksID; - } - - public void setBlocksID(String blocksID) { - this.blocksID = blocksID; - } - - public String getEscapeCharacter() { - return escapeCharacter; - } - - public void setEscapeCharacter(String escapeCharacter) { - this.escapeCharacter = escapeCharacter; - } - - public String getQuoteCharacter() { return quoteCharacter; } - - public void setQuoteCharacter(String quoteCharacter) { this.quoteCharacter = quoteCharacter; } - - public String getCommentCharacter() { return commentCharacter; } - - public void setCommentCharacter(String commentCharacter) { - this.commentCharacter = commentCharacter; - } - - public String getRddIteratorKey() { - return rddIteratorKey; - } - - public void setRddIteratorKey(String rddIteratorKey) { - this.rddIteratorKey = rddIteratorKey; - } - - public String getDateFormat() { return dateFormat; } - - public void setDateFormat(String dateFormat) { this.dateFormat = dateFormat; } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java deleted file mode 100644 index 412c1ff86b1..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java +++ /dev/null @@ -1,194 +0,0 @@ -/* - * 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.dataprocessor; - -import java.sql.Timestamp; -import java.util.List; - -public interface IDataProcessStatus { - - /** - * serial id - * - * @return - */ - int getDataloadstatusid(); - - /** - * @param dataloadstatusid - */ - void setDataloadstatusid(int dataloadstatusid); - - /** - * @return the createdTime - */ - Timestamp getCreatedTime(); - - /** - * @param createdTime the createdTime to set - */ - void setCreatedTime(Timestamp createdTime); - - /** - * return the description of the task - */ - String getDesc(); - - /** - * set the description of the task - */ - void setDesc(String desc); - - /** - * This method is used to get the Key for saving status of data loading. - * - * @return String - Key (databaseName + tableName + tableName). - */ - String getKey(); - - /** - * @param key - */ - void setKey(String key); - - /** - * To get the status of the data loading. - * - * @return String - Status - */ - String getStatus(); - - /** - * To set the status of the data loading. - */ - void setStatus(String status); - - /** - * Return task type - * 1- DATALOADING 2- RESTRUCTURE - */ - int getTaskType(); - - /** - * 1- DATALOADING 2- RESTRUCTURE - */ - void setTaskType(int taskType); - - /** - * @return the databaseName - */ - String getDatabaseName(); - - /** - * @param databaseName the databaseName to set - */ - void setDatabaseName(String databaseName); - - /** - * @return the tableName - */ - String getTableName(); - - /** - * @param tableName the tableName to set - */ - void setTableName(String tableName); - - /** - * @return the oldSchemaFilePath - */ - String getOldSchemaFilePath(); - - /** - * @param oldSchemaFilePath the oldSchemaFilePath to set - */ - void setOldSchemaFilePath(String oldSchemaFilePath); - - /** - * @return the newSchemaFilePath - */ - String getNewSchemaFilePath(); - - /** - * @param newSchemaFilePath the newSchemaFilePath to set - */ - void setNewSchemaFilePath(String newSchemaFilePath); - - /** - * @return the csvFilePath - */ - String getCsvFilePath(); - - /** - * @param csvFilePath the csvFilePath to set - */ - void setCsvFilePath(String csvFilePath); - - /** - * @return the dimTables - */ - String getDimTables(); - - /** - * @param dimTables the dimTables to set - */ - void setDimTables(String dimTables); - - /** - * @return the isFromPathApi - */ - boolean isFromPathApi(); - - /** - * @param isFromPathApi the isFromPathApi to set - */ - void setFromPathApi(boolean isFromPathApi); - - /** - * @return - */ - IDataProcessStatus makeCopy(); - - boolean isDirectLoad(); - - void setDirectLoad(boolean isDirectLoad); - - List getFilesToProcess(); - - void setFilesToProcess(List filesToProcess); - - String getCsvHeader(); - - void setCsvHeader(String csvHeader); - - String getCsvDelimiter(); - - void setCsvDelimiter(String csvDelimiter); - - String getBlocksID(); - - String getEscapeCharacter(); - - String getQuoteCharacter(); - - String getCommentCharacter(); - - String getRddIteratorKey(); - - String getDateFormat(); -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java deleted file mode 100644 index ea7945f29d9..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * 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.dataprocessor.queue.impl; - -import java.util.Comparator; - -import org.apache.carbondata.processing.dataprocessor.record.holder.DataProcessorRecordHolder; - -public class RecordComparator implements Comparator { - - @Override public int compare(DataProcessorRecordHolder o1, DataProcessorRecordHolder o2) { - - if (o1 == null) { - return -1; - } - if (o2 == null) { - return 1; - } - - if (o1.getSeqNumber() < o2.getSeqNumber()) { - return -1; - } else if (o1.getSeqNumber() > o2.getSeqNumber()) { - return 1; - } - return 0; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java deleted file mode 100644 index 68ca84c467a..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.dataprocessor.record.holder; - -/** - * Data processor for the record. - */ -public class DataProcessorRecordHolder { - - private int seqNumber; - - public DataProcessorRecordHolder(int size, int seqNumber) { - this.seqNumber = seqNumber; - } - - /** - * Returns the sequence number. - */ - public int getSeqNumber() { - return seqNumber; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java index 442c0f0ce69..02ceb06110a 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java @@ -23,14 +23,11 @@ import java.util.ArrayList; import java.util.List; -import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.devapi.DictionaryGenerationException; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; import org.apache.carbondata.processing.newflow.complexobjects.ArrayObject; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen; -import org.pentaho.di.core.exception.KettleException; /** * Array DataType stateless object used in data loading @@ -143,32 +140,6 @@ public void setSurrogateIndex(int surrIndex) { } - /* - * parse string and generate surrogate - */ - @Override - public void parseStringAndWriteByteArray(String tableName, String inputString, - String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream, - CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException { - - if (inputString == null || "null".equals(inputString) || "".equals(inputString) || - CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(inputString)) { - dataOutputStream.writeInt(1); - children.parseStringAndWriteByteArray(tableName, - CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream, - surrogateKeyGen); - } else { - String[] splitInput = inputString.split(delimiter[delimiterIndex], -1); - dataOutputStream.writeInt(splitInput.length); - delimiterIndex = - (delimiter.length - 1) == delimiterIndex ? delimiterIndex : delimiterIndex + 1; - for (String eachInput : splitInput) { - children.parseStringAndWriteByteArray(tableName, eachInput, delimiter, delimiterIndex, - dataOutputStream, surrogateKeyGen); - } - } - } - @Override public void writeByteArray(ArrayObject input, DataOutputStream dataOutputStream) throws IOException, DictionaryGenerationException { diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java index 91caf7af203..6b54d2ddf9e 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java @@ -26,9 +26,6 @@ import org.apache.carbondata.core.devapi.DictionaryGenerationException; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen; - -import org.pentaho.di.core.exception.KettleException; /** * Generic DataType interface which will be used while data loading for complex types like Array & @@ -56,21 +53,6 @@ public interface GenericDataType { */ void getAllPrimitiveChildren(List primitiveChild); - /** - * Split raw csv data into seperate column using delimiter and generate surrogate key - * @param tableName - * @param inputString - * @param delimiter - * @param delimiterIndex - * @param dataOutputStream - * @param surrogateKeyGen - * @throws KettleException - * @throws IOException - */ - void parseStringAndWriteByteArray(String tableName, String inputString, String[] delimiter, - int delimiterIndex, DataOutputStream dataOutputStream, - CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException; - /** * writes to byte stream * @param dataOutputStream diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java index 105f5f4545d..e7e48e9acce 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java @@ -44,9 +44,6 @@ import org.apache.carbondata.processing.newflow.dictionary.DictionaryServerClientDictionary; import org.apache.carbondata.processing.newflow.dictionary.DirectDictionary; import org.apache.carbondata.processing.newflow.dictionary.PreCreatedDictionary; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen; - -import org.pentaho.di.core.exception.KettleException; /** * Primitive DataType stateless object used in data loading @@ -218,28 +215,6 @@ public void setSurrogateIndex(int surrIndex) { index = surrIndex; } - /* - * parse string and generate surrogate - */ - @Override public void parseStringAndWriteByteArray(String tableName, String inputString, - String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream, - CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException { - String parsedValue = DataTypeUtil.parseValue(inputString, - surrogateKeyGen.getDimensionOrdinalToDimensionMapping()[dimensionOrdinal]); - Integer surrogateKey = null; - if (null == parsedValue) { - surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - } else { - surrogateKey = surrogateKeyGen - .generateSurrogateKeys(parsedValue, tableName + CarbonCommonConstants.UNDERSCORE + name, - this.getColumnId()); - if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) { - surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - } - } - dataOutputStream.writeInt(surrogateKey); - } - @Override public void writeByteArray(Object input, DataOutputStream dataOutputStream) throws IOException, DictionaryGenerationException { String parsedValue = diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java index a13100469b9..a61144ee2e5 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java +++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java @@ -23,14 +23,10 @@ import java.util.ArrayList; import java.util.List; -import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.devapi.DictionaryGenerationException; import org.apache.carbondata.core.keygenerator.KeyGenException; import org.apache.carbondata.core.keygenerator.KeyGenerator; import org.apache.carbondata.processing.newflow.complexobjects.StructObject; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen; - -import org.pentaho.di.core.exception.KettleException; /** * Struct DataType stateless object used in data loading @@ -144,40 +140,6 @@ public void setSurrogateIndex(int surrIndex) { } - /* - * parse string and generate surrogate - */ - @Override - public void parseStringAndWriteByteArray(String tableName, String inputString, - String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream, - CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException { - if (inputString == null || "null".equals(inputString)) { - // Indicates null array - dataOutputStream.writeInt(children.size()); - // For other children elements which dont have data, write empty - for (int i = 0; i < children.size(); i++) { - children.get(i).parseStringAndWriteByteArray(tableName, - CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream, - surrogateKeyGen); - } - } else { - String[] splitInput = inputString.split(delimiter[delimiterIndex], -1); - dataOutputStream.writeInt(children.size()); - delimiterIndex = - (delimiter.length - 1) == delimiterIndex ? delimiterIndex : delimiterIndex + 1; - for (int i = 0; i < splitInput.length && i < children.size(); i++) { - children.get(i).parseStringAndWriteByteArray(tableName, splitInput[i], delimiter, - delimiterIndex, dataOutputStream, surrogateKeyGen); - } - // For other children elements which dont have data, write empty - for (int i = splitInput.length; i < children.size(); i++) { - children.get(i).parseStringAndWriteByteArray(tableName, - CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream, - surrogateKeyGen); - } - } - } - @Override public void writeByteArray(StructObject input, DataOutputStream dataOutputStream) throws IOException, DictionaryGenerationException { dataOutputStream.writeInt(children.size()); diff --git a/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java b/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java deleted file mode 100644 index 7552e6af439..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.exception; - -import java.util.Locale; - -public class CarbonDataProcessorException extends Exception { - - /** - * default serial version ID. - */ - private static final long serialVersionUID = 1L; - - /** - * The Error message. - */ - private String msg = ""; - - /** - * Constructor - * - * @param msg The error message for this exception. - */ - public CarbonDataProcessorException(String msg) { - super(msg); - this.msg = msg; - } - - /** - * Constructor - * - * @param msg The error message for this exception. - */ - public CarbonDataProcessorException(String msg, Throwable t) { - super(msg, t); - this.msg = msg; - } - - /** - * This method is used to get the localized message. - * - * @param locale - A Locale object represents a specific geographical, - * political, or cultural region. - * @return - Localized error message. - */ - public String getLocalizedMessage(Locale locale) { - return ""; - } - - /** - * getLocalizedMessage - */ - @Override public String getLocalizedMessage() { - return super.getLocalizedMessage(); - } - - /** - * getMessage - */ - public String getMessage() { - return this.msg; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java deleted file mode 100644 index 2876eb8a538..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java +++ /dev/null @@ -1,965 +0,0 @@ -/* - * 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.graphgenerator; - -import java.io.DataOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.UnsupportedEncodingException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.metadata.CarbonMetadata; -import org.apache.carbondata.core.metadata.encoder.Encoding; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.processing.api.dataloader.DataLoadModel; -import org.apache.carbondata.processing.api.dataloader.SchemaInfo; -import org.apache.carbondata.processing.csvreaderstep.BlockDetails; -import org.apache.carbondata.processing.csvreaderstep.CsvInputMeta; -import org.apache.carbondata.processing.graphgenerator.configuration.GraphConfigurationInfo; -import org.apache.carbondata.processing.mdkeygen.MDKeyGenStepMeta; -import org.apache.carbondata.processing.merger.step.CarbonSliceMergerStepMeta; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; -import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper; -import org.apache.carbondata.processing.sortandgroupby.sortdatastep.SortKeyStepMeta; -import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedSeqGenMeta; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; -import org.apache.carbondata.processing.util.CarbonSchemaParser; -import org.apache.carbondata.processing.util.NonDictionaryUtil; - -import org.pentaho.di.core.KettleEnvironment; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.util.EnvUtil; -import org.pentaho.di.trans.TransHopMeta; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.pentaho.di.trans.steps.hadoopfileinput.HadoopFileInputMeta; -import org.pentaho.di.trans.steps.selectvalues.SelectMetadataChange; -import org.pentaho.di.trans.steps.selectvalues.SelectValuesMeta; -import org.pentaho.di.trans.steps.tableinput.TableInputMeta; - -public class GraphGenerator { - - public static final HashMap blockInfo = new HashMap<>(); - /** - * DEFAUL_BLOCKLET_SIZE - */ - private static final String DEFAUL_BLOCKLET_SIZE = "8192"; - /** - * DEFAULE_MAX_BLOCKLET_IN_FILE - */ - private static final String DEFAULE_MAX_BLOCKLET_IN_FILE = "100"; - /** - * DEFAULT_NUMBER_CORES - */ - private static final String DEFAULT_NUMBER_CORES = "2"; - /** - * DEFAULT_BATCH_SIZE - */ - private static final String DEFAULT_BATCH_SIZE = "1000"; - /** - * DEFAULT_SORT_SIZE - */ - private static final String DEFAULT_SORT_SIZE = "100000"; - /** - * drivers - */ - private static final Map DRIVERS; - /** - * Comment for LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(GraphGenerator.class.getName()); - /** - * kettleInitialized - */ - private static boolean kettleInitialized = false; - - static { - - DRIVERS = new HashMap(1); - - DRIVERS.put("oracle.jdbc.OracleDriver", CarbonCommonConstants.TYPE_ORACLE); - DRIVERS.put("com.mysql.jdbc.Driver", CarbonCommonConstants.TYPE_MYSQL); - DRIVERS.put("org.gjt.mm.mysql.Driver", CarbonCommonConstants.TYPE_MYSQL); - DRIVERS.put("com.microsoft.sqlserver.jdbc.SQLServerDriver", CarbonCommonConstants.TYPE_MSSQL); - DRIVERS.put("com.sybase.jdbc3.jdbc.SybDriver", CarbonCommonConstants.TYPE_SYBASE); - } - - /** - * OUTPUT_LOCATION - */ - private String outputLocation = ""; - /** - * xAxixLocation - */ - private int xAxixLocation = 50; - /** - * yAxixLocation - */ - private int yAxixLocation = 100; - /** - * databaseName - */ - private String databaseName; - /** - * table - */ - // private Table table; - /** - * instance - */ - private CarbonProperties instance; - /** - * schemaInfo - */ - private SchemaInfo schemaInfo; - /** - * Table name - */ - private String tableName; - /** - * Is CSV Load request - */ - private boolean isCSVLoad; - /** - * Modified dimension - */ - private String[] modifiedDimension; - /** - * isAutoAggRequest - */ - private boolean isAutoAggRequest; - /** - * schema - */ - private CarbonDataLoadSchema carbonDataLoadSchema; - /** - * isUpdateMemberRequest - */ - private boolean isUpdateMemberRequest; - /** - * If the CSV file is present in HDFS? - */ - private boolean isHDFSReadMode; - /** - * partitionID - */ - private String partitionID; - private boolean isColumnar; - private String factTableName; - private String factStoreLocation; - private String blocksID; - private String escapeCharacter; - private String quoteCharacter; - private String commentCharacter; - private String dateFormat; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - /** - * load Id - */ - private String segmentId; - /** - * new load start time - */ - private String factTimeStamp; - /** - * max number of columns configured by user to be parsed in a row - */ - private String maxColumns; - - private String rddIteratorKey; - - public GraphGenerator(DataLoadModel dataLoadModel, String partitionID, String factStoreLocation, - CarbonDataLoadSchema carbonDataLoadSchema, String segmentId) { - CarbonMetadata.getInstance().addCarbonTable(carbonDataLoadSchema.getCarbonTable()); - this.schemaInfo = dataLoadModel.getSchemaInfo(); - this.tableName = dataLoadModel.getTableName(); - this.isCSVLoad = dataLoadModel.isCsvLoad(); - this.isAutoAggRequest = schemaInfo.isAutoAggregateRequest(); - this.carbonDataLoadSchema = carbonDataLoadSchema; - this.databaseName = carbonDataLoadSchema.getCarbonTable().getDatabaseName(); - this.partitionID = partitionID; - this.factStoreLocation = factStoreLocation; - this.isColumnar = Boolean.parseBoolean(CarbonCommonConstants.IS_COLUMNAR_STORAGE_DEFAULTVALUE); - this.blocksID = dataLoadModel.getBlocksID(); - this.taskNo = dataLoadModel.getTaskNo(); - this.quoteCharacter = dataLoadModel.getQuoteCharacter(); - this.commentCharacter = dataLoadModel.getCommentCharacter(); - this.dateFormat = dataLoadModel.getDateFormat(); - this.factTimeStamp = dataLoadModel.getFactTimeStamp(); - this.segmentId = segmentId; - this.escapeCharacter = dataLoadModel.getEscapeCharacter(); - this.maxColumns = dataLoadModel.getMaxColumns(); - initialise(); - LOGGER.info("************* Is Columnar Storage" + isColumnar); - } - - public GraphGenerator(DataLoadModel dataLoadModel, String partitionID, String factStoreLocation, - CarbonDataLoadSchema carbonDataLoadSchema, String segmentId, String outputLocation) { - this(dataLoadModel, partitionID, factStoreLocation, carbonDataLoadSchema, segmentId); - this.outputLocation = outputLocation; - this.rddIteratorKey = dataLoadModel.getRddIteratorKey(); - } - - /** - * Generate the graph file ... - * - * @param transMeta - * @param graphFile - * @throws KettleException - */ - private static void generateGraphFile(TransMeta transMeta, String graphFile) - throws GraphGeneratorException { - // - DataOutputStream dos = null; - try { - String xml = transMeta.getXML(); - dos = new DataOutputStream(new FileOutputStream(new File(graphFile))); - dos.write(xml.getBytes(CarbonCommonConstants.DEFAULT_CHARSET)); - } catch (KettleException kettelException) { - throw new GraphGeneratorException("Error while getting the graph XML", kettelException); - } - // - catch (FileNotFoundException e) { - throw new GraphGeneratorException("Unable to find the graph fileL", e); - } - // - catch (UnsupportedEncodingException ue) { - throw new GraphGeneratorException("Error while Converting the graph xml string to bytes", ue); - } - // - catch (IOException ioe) { - throw new GraphGeneratorException("Error while writing the graph file", ioe); - } finally { - // - if (dos != null) { - try { - dos.close(); - } catch (IOException e) { - e.getMessage(); - } - } - } - } - - private void initialise() { - this.instance = CarbonProperties.getInstance(); - //TO-DO need to take care while supporting aggregate table using new schema. - //aggregateTable = CarbonSchemaParser.getAggregateTable(table, schema); - this.factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName(); - } - - public void generateGraph() throws GraphGeneratorException { - validateAndInitialiseKettelEngine(); - GraphConfigurationInfo graphConfigInfoForFact = getGraphConfigInfoForFact(carbonDataLoadSchema); - generateGraph(graphConfigInfoForFact, graphConfigInfoForFact.getTableName() + ": Graph", - isCSVLoad, graphConfigInfoForFact); - } - - private void validateAndInitialiseKettelEngine() throws GraphGeneratorException { - File file = new File( - outputLocation + File.separator + schemaInfo.getDatabaseName() + File.separator - + this.tableName + File.separator + this.segmentId + File.separator + this.taskNo - + File.separator); - boolean isDirCreated = false; - if (!file.exists()) { - isDirCreated = file.mkdirs(); - - if (!isDirCreated) { - LOGGER.error( - "Unable to create directory or directory already exist" + file.getAbsolutePath()); - throw new GraphGeneratorException("INTERNAL_SYSTEM_ERROR"); - } - } - - synchronized (DRIVERS) { - try { - if (!kettleInitialized) { - EnvUtil.environmentInit(); - KettleEnvironment.init(); - kettleInitialized = true; - } - } catch (KettleException kettlExp) { - LOGGER.error(kettlExp); - throw new GraphGeneratorException("Error While Initializing the Kettel Engine ", kettlExp); - } - } - } - - private void generateGraph(GraphConfigurationInfo configurationInfo, String transName, - boolean isCSV, GraphConfigurationInfo configurationInfoForFact) - throws GraphGeneratorException { - - TransMeta trans = new TransMeta(); - trans.setName(transName); - - if (!isCSV) { - trans.addDatabase(getDatabaseMeta(configurationInfo)); - } - - trans.setSizeRowset(Integer.parseInt(instance - .getProperty(CarbonCommonConstants.GRAPH_ROWSET_SIZE, - CarbonCommonConstants.GRAPH_ROWSET_SIZE_DEFAULT))); - - StepMeta inputStep = null; - StepMeta carbonSurrogateKeyStep = null; - StepMeta selectValueToChangeTheDataType = null; - - // get all step - if (isCSV) { - if (isHDFSReadMode) { - inputStep = getHadoopInputStep(configurationInfo); - } else { - inputStep = getCSVInputStep(configurationInfo); - } - } else { - inputStep = getTableInputStep(configurationInfo); - selectValueToChangeTheDataType = getSelectValueToChangeTheDataType(configurationInfo, 1); - } - carbonSurrogateKeyStep = getCarbonCSVBasedSurrogateKeyStep(configurationInfo); - StepMeta sortStep = getSortStep(configurationInfo); - StepMeta carbonMDKeyStep = getMDKeyStep(configurationInfo); - StepMeta carbonSliceMergerStep = null; - carbonSliceMergerStep = getSliceMeregerStep(configurationInfo, configurationInfoForFact); - - // add all steps to trans - trans.addStep(inputStep); - - if (!isCSV) { - trans.addStep(selectValueToChangeTheDataType); - } - - trans.addStep(carbonSurrogateKeyStep); - trans.addStep(sortStep); - trans.addStep(carbonMDKeyStep); - - trans.addStep(carbonSliceMergerStep); - TransHopMeta inputStepToSelectValueHop = null; - TransHopMeta tableInputToSelectValue = null; - - if (isCSV) { - inputStepToSelectValueHop = new TransHopMeta(inputStep, carbonSurrogateKeyStep); - } else { - inputStepToSelectValueHop = new TransHopMeta(inputStep, selectValueToChangeTheDataType); - tableInputToSelectValue = - new TransHopMeta(selectValueToChangeTheDataType, carbonSurrogateKeyStep); - } - - // create hop - TransHopMeta surrogateKeyToSortHop = new TransHopMeta(carbonSurrogateKeyStep, sortStep); - TransHopMeta sortToMDKeyHop = new TransHopMeta(sortStep, carbonMDKeyStep); - TransHopMeta mdkeyToSliceMerger = null; - mdkeyToSliceMerger = new TransHopMeta(carbonMDKeyStep, carbonSliceMergerStep); - - if (isCSV) { - trans.addTransHop(inputStepToSelectValueHop); - } else { - trans.addTransHop(inputStepToSelectValueHop); - trans.addTransHop(tableInputToSelectValue); - } - - trans.addTransHop(surrogateKeyToSortHop); - trans.addTransHop(sortToMDKeyHop); - trans.addTransHop(mdkeyToSliceMerger); - - String graphFilePath = - outputLocation + File.separator + schemaInfo.getDatabaseName() + File.separator - + this.tableName + File.separator + segmentId + File.separator + this.taskNo - + File.separator + this.tableName + ".ktr"; - generateGraphFile(trans, graphFilePath); - } - - private StepMeta getHadoopInputStep(GraphConfigurationInfo graphConfiguration) - throws GraphGeneratorException { - HadoopFileInputMeta fileInputMeta = new HadoopFileInputMeta(); - fileInputMeta.setFilenameField("filename"); - fileInputMeta.setFileName(new String[] { "${csvInputFilePath}" }); - fileInputMeta.setDefault(); - fileInputMeta.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET); - fileInputMeta.setEnclosure("\""); - fileInputMeta.setHeader(true); - fileInputMeta.setSeparator(","); - fileInputMeta.setAcceptingFilenames(true); - fileInputMeta.setAcceptingStepName("getFileNames"); - fileInputMeta.setFileFormat("mixed"); - StepMeta csvDataStep = new StepMeta("HadoopFileInputPlugin", (StepMetaInterface) fileInputMeta); - csvDataStep.setLocation(100, 100); - int copies = Integer.parseInt(instance.getProperty(CarbonCommonConstants.NUM_CORES_LOADING, - CarbonCommonConstants.DEFAULT_NUMBER_CORES)); - if (copies > 1) { - csvDataStep.setCopies(4); - } - csvDataStep.setDraw(true); - csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT); - - return csvDataStep; - } - - private StepMeta getCSVInputStep(GraphConfigurationInfo graphConfiguration) - throws GraphGeneratorException { - CsvInputMeta csvInputMeta = new CsvInputMeta(); - // Init the Filename... - csvInputMeta.setFilename("${csvInputFilePath}"); - csvInputMeta.setDefault(); - csvInputMeta.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET); - csvInputMeta.setEnclosure("\""); - csvInputMeta.setHeaderPresent(true); - csvInputMeta.setMaxColumns(maxColumns); - StepMeta csvDataStep = - new StepMeta(GraphGeneratorConstants.CSV_INPUT, (StepMetaInterface) csvInputMeta); - csvDataStep.setLocation(100, 100); - csvInputMeta.setFilenameField("filename"); - csvInputMeta.setLazyConversionActive(false); - csvInputMeta.setBufferSize(instance.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE, - CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT)); - //set blocks info id - csvInputMeta.setBlocksID(this.blocksID); - csvInputMeta.setPartitionID(this.partitionID); - csvInputMeta.setEscapeCharacter(this.escapeCharacter); - csvInputMeta.setQuoteCharacter(this.quoteCharacter); - csvInputMeta.setCommentCharacter(this.commentCharacter); - csvInputMeta.setRddIteratorKey(this.rddIteratorKey == null ? "" : this.rddIteratorKey); - csvDataStep.setDraw(true); - csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT); - - return csvDataStep; - } - - private StepMeta getSliceMeregerStep(GraphConfigurationInfo configurationInfo, - GraphConfigurationInfo graphjConfigurationForFact) { - CarbonSliceMergerStepMeta sliceMerger = new CarbonSliceMergerStepMeta(); - sliceMerger.setDefault(); - sliceMerger.setPartitionID(partitionID); - sliceMerger.setSegmentId(segmentId); - sliceMerger.setTaskNo(taskNo); - sliceMerger.setHeirAndKeySize(configurationInfo.getHeirAndKeySizeString()); - sliceMerger.setMdkeySize(configurationInfo.getMdkeySize()); - sliceMerger.setMeasureCount(configurationInfo.getMeasureCount()); - sliceMerger.setTabelName(configurationInfo.getTableName()); - sliceMerger.setTableName(schemaInfo.getTableName()); - sliceMerger.setDatabaseName(schemaInfo.getDatabaseName()); - sliceMerger.setGroupByEnabled(isAutoAggRequest + ""); - if (isAutoAggRequest) { - String[] aggType = configurationInfo.getAggType(); - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < aggType.length - 1; i++) { - if (aggType[i].equals(CarbonCommonConstants.COUNT)) { - builder.append(CarbonCommonConstants.SUM); - } else { - builder.append(aggType[i]); - } - builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - builder.append(aggType[aggType.length - 1]); - sliceMerger.setAggregatorString(builder.toString()); - String[] aggClass = configurationInfo.getAggClass(); - builder = new StringBuilder(); - for (int i = 0; i < aggClass.length - 1; i++) { - builder.append(aggClass[i]); - builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - builder.append(aggClass[aggClass.length - 1]); - sliceMerger.setAggregatorClassString(builder.toString()); - } else { - sliceMerger.setAggregatorClassString(CarbonCommonConstants.HASH_SPC_CHARACTER); - sliceMerger.setAggregatorString(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - sliceMerger.setFactDimLensString(""); - sliceMerger.setLevelAnddataTypeString(configurationInfo.getLevelAnddataType()); - StepMeta sliceMergerMeta = - new StepMeta(GraphGeneratorConstants.CARBON_SLICE_MERGER + configurationInfo.getTableName(), - (StepMetaInterface) sliceMerger); - sliceMergerMeta.setStepID(GraphGeneratorConstants.CARBON_SLICE_MERGER_ID); - xAxixLocation += 120; - sliceMergerMeta.setLocation(xAxixLocation, yAxixLocation); - sliceMergerMeta.setDraw(true); - sliceMergerMeta.setDescription( - "SliceMerger: " + GraphGeneratorConstants.CARBON_SLICE_MERGER + configurationInfo - .getTableName()); - return sliceMergerMeta; - } - - private DatabaseMeta getDatabaseMeta(GraphConfigurationInfo configurationInfo) - throws GraphGeneratorException { - return new DatabaseMeta(); - } - - private StepMeta getTableInputStep(GraphConfigurationInfo configurationInfo) - throws GraphGeneratorException { - TableInputMeta tableInput = new TableInputMeta(); - tableInput.setDatabaseMeta(getDatabaseMeta(configurationInfo)); - tableInput.setSQL(configurationInfo.getTableInputSqlQuery()); - // - StepMeta tableInputStep = - new StepMeta(GraphGeneratorConstants.TABLE_INPUT, (StepMetaInterface) tableInput); - xAxixLocation += 120; - tableInputStep.setLocation(xAxixLocation, yAxixLocation); - // - tableInputStep.setDraw(true); - tableInputStep - .setDescription("Read Data From Fact Table: " + GraphGeneratorConstants.TABLE_INPUT); - - return tableInputStep; - } - - private StepMeta getCarbonCSVBasedSurrogateKeyStep(GraphConfigurationInfo graphConfiguration) { - // - CarbonCSVBasedSeqGenMeta seqMeta = new CarbonCSVBasedSeqGenMeta(); - seqMeta.setPartitionID(partitionID); - seqMeta.setSegmentId(segmentId); - seqMeta.setTaskNo(taskNo); - seqMeta.setCarbondim(graphConfiguration.getDimensionString()); - seqMeta.setComplexTypeString(graphConfiguration.getComplexTypeString()); - seqMeta.setColumnPropertiesString(graphConfiguration.getColumnPropertiesString()); - seqMeta.setBatchSize(Integer.parseInt(graphConfiguration.getBatchSize())); - seqMeta.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims()); - seqMeta.setDimensionColumnsDataType(graphConfiguration.getDimensionColumnsDataType()); - seqMeta.setTableName(schemaInfo.getTableName()); - seqMeta.setDatabaseName(schemaInfo.getDatabaseName()); - seqMeta.setComplexDelimiterLevel1(schemaInfo.getComplexDelimiterLevel1()); - seqMeta.setComplexDelimiterLevel2(schemaInfo.getComplexDelimiterLevel2()); - seqMeta.setCarbonmsr(graphConfiguration.getMeasuresString()); - seqMeta.setCarbonProps(graphConfiguration.getPropertiesString()); - seqMeta.setCarbonhier(graphConfiguration.getHiersString()); - seqMeta.setCarbonhierColumn(graphConfiguration.getHierColumnString()); - seqMeta.setDimensionColumnIds(graphConfiguration.getDimensionColumnIds()); - seqMeta.setMetaMetaHeirSQLQueries(graphConfiguration.getDimensionSqlQuery()); - seqMeta.setColumnAndTableNameColumnMapForAggString( - graphConfiguration.getColumnAndTableNameColumnMapForAgg()); - seqMeta.setForgienKeyPrimayKeyString(graphConfiguration.getForgienKeyAndPrimaryKeyMapString()); - seqMeta.setTableName(graphConfiguration.getTableName()); - seqMeta.setDateFormat(dateFormat); - seqMeta.setModifiedDimension(modifiedDimension); - seqMeta.setForeignKeyHierarchyString(graphConfiguration.getForeignKeyHierarchyString()); - seqMeta.setPrimaryKeysString(graphConfiguration.getPrimaryKeyString()); - seqMeta.setCarbonMeasureNames(graphConfiguration.getMeasureNamesString()); - seqMeta.setHeirNadDimsLensString(graphConfiguration.getHeirAndDimLens()); - seqMeta.setActualDimNames(graphConfiguration.getActualDimensionColumns()); - seqMeta.setNormHiers(graphConfiguration.getNormHiers()); - seqMeta.setHeirKeySize(graphConfiguration.getHeirAndKeySizeString()); - seqMeta.setColumnSchemaDetails(graphConfiguration.getColumnSchemaDetails().toString()); - seqMeta.setTableOption(graphConfiguration.getTableOptionWrapper().toString()); - String[] aggType = graphConfiguration.getAggType(); - StringBuilder builder = new StringBuilder(); - for (int i = 0; i < aggType.length; i++) { - builder.append(aggType[i]); - builder.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER); - } - seqMeta.setMsrAggregatorString(builder.toString()); - - seqMeta.setDriverClass(graphConfiguration.getDriverclass()); - seqMeta.setConnectionURL(graphConfiguration.getConnectionUrl()); - seqMeta.setUserName(graphConfiguration.getUsername()); - seqMeta.setPassword(graphConfiguration.getPassword()); - seqMeta.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo()); - seqMeta.setDenormColumNames(graphConfiguration.getDenormColumns()); - seqMeta.setAggregate(graphConfiguration.isAGG()); - seqMeta.setTableNames(graphConfiguration.getDimensionTableNames()); - StepMeta mdkeyStepMeta = new StepMeta(GraphGeneratorConstants.CARBON_SURROGATE_KEY_GENERATOR, - (StepMetaInterface) seqMeta); - mdkeyStepMeta.setStepID(GraphGeneratorConstants.CARBON_CSV_BASED_SURROAGATEGEN_ID); - xAxixLocation += 120; - // - mdkeyStepMeta.setLocation(xAxixLocation, yAxixLocation); - mdkeyStepMeta.setDraw(true); - mdkeyStepMeta.setDescription("Generate Surrogate For Table Data: " - + GraphGeneratorConstants.CARBON_SURROGATE_KEY_GENERATOR); - return mdkeyStepMeta; - } - - private StepMeta getMDKeyStep(GraphConfigurationInfo graphConfiguration) { - MDKeyGenStepMeta carbonMdKey = new MDKeyGenStepMeta(); - carbonMdKey.setIsUseInvertedIndex( - NonDictionaryUtil.convertBooleanArrToString(graphConfiguration.getIsUseInvertedIndex())); - carbonMdKey.setPartitionID(partitionID); - carbonMdKey.setSegmentId(segmentId); - carbonMdKey.setNumberOfCores(graphConfiguration.getNumberOfCores()); - carbonMdKey.setTableName(graphConfiguration.getTableName()); - carbonMdKey.setDatabaseName(schemaInfo.getDatabaseName()); - carbonMdKey.setTableName(schemaInfo.getTableName()); - carbonMdKey.setComplexTypeString(graphConfiguration.getComplexTypeString()); - carbonMdKey.setAggregateLevels(CarbonDataProcessorUtil - .getLevelCardinalitiesString(graphConfiguration.getDimCardinalities(), - graphConfiguration.getDimensions())); - carbonMdKey.setNoDictionaryDimsMapping(NonDictionaryUtil - .convertBooleanArrToString(graphConfiguration.getIsNoDictionaryDimMapping())); - carbonMdKey.setMeasureCount(graphConfiguration.getMeasureCount() + ""); - carbonMdKey.setColumnGroupsString(graphConfiguration.getColumnGroupsString()); - carbonMdKey.setDimensionCount(graphConfiguration.getActualDims().length + ""); - carbonMdKey.setComplexDimsCount(graphConfiguration.getComplexTypeString().isEmpty() ? - "0" : - graphConfiguration.getComplexTypeString() - .split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER).length + ""); - carbonMdKey.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo()); - carbonMdKey.setTaskNo(taskNo); - carbonMdKey.setFactTimeStamp(factTimeStamp); - StepMeta mdkeyStepMeta = - new StepMeta(GraphGeneratorConstants.MDKEY_GENERATOR + graphConfiguration.getTableName(), - (StepMetaInterface) carbonMdKey); - mdkeyStepMeta - .setName(GraphGeneratorConstants.MDKEY_GENERATOR_ID + graphConfiguration.getTableName()); - mdkeyStepMeta.setStepID(GraphGeneratorConstants.MDKEY_GENERATOR_ID); - // - xAxixLocation += 120; - mdkeyStepMeta.setLocation(xAxixLocation, yAxixLocation); - mdkeyStepMeta.setDraw(true); - mdkeyStepMeta.setDescription( - "Generate MDKey For Table Data: " + GraphGeneratorConstants.MDKEY_GENERATOR - + graphConfiguration.getTableName()); - carbonMdKey.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims()); - - return mdkeyStepMeta; - } - - private StepMeta getSelectValueToChangeTheDataType(GraphConfigurationInfo graphConfiguration, - int counter) { - // - SelectValuesMeta selectValues = new SelectValuesMeta(); - selectValues.allocate(0, 0, 0); - StepMeta selectValueMeta = new StepMeta( - GraphGeneratorConstants.SELECT_REQUIRED_VALUE + "Change Dimension And Measure DataType" - + System.currentTimeMillis() + counter, (StepMetaInterface) selectValues); - xAxixLocation += 120; - selectValueMeta.setName("SelectValueToChangeChangeData"); - selectValueMeta.setLocation(xAxixLocation, yAxixLocation); - selectValueMeta.setDraw(true); - selectValueMeta.setDescription( - "Change The Data Type For Measures: " + GraphGeneratorConstants.SELECT_REQUIRED_VALUE); - - String inputQuery = graphConfiguration.getTableInputSqlQuery(); - String[] columns = parseQueryAndReturnColumns(inputQuery); - - SelectMetadataChange[] changeMeta = new SelectMetadataChange[columns.length]; - Map measureDatatypeMap = - getMeasureDatatypeMap(graphConfiguration.getMeasureDataTypeInfo()); - String[] measures = graphConfiguration.getMeasures(); - String dimensionString = graphConfiguration.getActualDimensionColumns(); - String[] dimension = dimensionString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < columns.length; i++) { - changeMeta[i] = new SelectMetadataChange(selectValues); - changeMeta[i].setName(columns[i]); - changeMeta[i].setType(2); - if (isMeasureColumn(measures, columns[i]) && isNotDimesnionColumn(dimension, columns[i])) { - Boolean isString = measureDatatypeMap.get(columns[i]); - if (isString != null && isString) { - changeMeta[i].setType(2); - } else { - changeMeta[i].setType(6); - } - } - changeMeta[i].setStorageType(0); - } - // - selectValues.setMeta(changeMeta); - return selectValueMeta; - } - - private boolean isMeasureColumn(String[] measures, String column) { - for (int i = 0; i < measures.length; i++) { - if (measures[i].equals(column)) { - return true; - } - } - return false; - } - - private boolean isNotDimesnionColumn(String[] dimension, String column) { - for (int i = 0; i < dimension.length; i++) { - if (dimension[i].equals(column)) { - return false; - } - } - return true; - } - - private Map getMeasureDatatypeMap(String measureDataType) { - if (measureDataType == null || "".equals(measureDataType)) { - return new HashMap(1); - } - Map resultMap = new HashMap(1); - - String[] measures = measureDataType.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - String[] measureValue = null; - for (int i = 0; i < measures.length; i++) { - measureValue = measures[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - resultMap.put(measureValue[0], Boolean.valueOf(measureValue[1])); - } - return resultMap; - } - - /** - * @param inputQuery - * @return - */ - private String[] parseQueryAndReturnColumns(String inputQuery) { - Set cols = new LinkedHashSet(); - String columnString = - inputQuery.substring(inputQuery.indexOf("SELECT") + 6, inputQuery.indexOf("FROM")); - String[] columns = columnString.split(","); - for (int i = 0; i < columns.length; i++) { - if (columns[i].indexOf("\"") > -1) { - columns[i] = columns[i].replace("\"", ""); - if (columns[i].contains(".")) { - columns[i] = columns[i].split("\\.")[1]; - } - } - - cols.add(columns[i].replaceAll(System.getProperty("line.separator"), "").trim()); - } - return cols.toArray(new String[cols.size()]); - } - - private StepMeta getSortStep(GraphConfigurationInfo graphConfiguration) - throws GraphGeneratorException { - String[] actualMeasures = graphConfiguration.getMeasures(); - - SortKeyStepMeta sortRowsMeta = new SortKeyStepMeta(); - sortRowsMeta.setPartitionID(partitionID); - sortRowsMeta.setSegmentId(segmentId); - sortRowsMeta.setTaskNo(taskNo); - sortRowsMeta.setTabelName(graphConfiguration.getTableName()); - sortRowsMeta.setTableName(schemaInfo.getTableName()); - sortRowsMeta.setDatabaseName(schemaInfo.getDatabaseName()); - sortRowsMeta.setOutputRowSize(actualMeasures.length + 1 + ""); - sortRowsMeta.setDimensionCount(graphConfiguration.getDimensions().length + ""); - sortRowsMeta.setComplexDimensionCount(graphConfiguration.getComplexTypeString().isEmpty() ? - "0" : - graphConfiguration.getComplexTypeString() - .split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER).length + ""); - sortRowsMeta.setIsUpdateMemberRequest(isUpdateMemberRequest + ""); - sortRowsMeta.setMeasureCount(graphConfiguration.getMeasureCount() + ""); - sortRowsMeta.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims()); - sortRowsMeta.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo()); - sortRowsMeta.setNoDictionaryDimsMapping(NonDictionaryUtil - .convertBooleanArrToString(graphConfiguration.getIsNoDictionaryDimMapping())); - - StepMeta sortRowsStep = new StepMeta( - GraphGeneratorConstants.SORT_KEY_AND_GROUPBY + graphConfiguration.getTableName(), - (StepMetaInterface) sortRowsMeta); - - xAxixLocation += 120; - sortRowsStep.setDraw(true); - sortRowsStep.setLocation(xAxixLocation, yAxixLocation); - sortRowsStep.setStepID(GraphGeneratorConstants.SORTKEY_ID); - sortRowsStep.setDescription( - "Sort Key: " + GraphGeneratorConstants.SORT_KEY + graphConfiguration.getTableName()); - sortRowsStep.setName( - "Sort Key: " + GraphGeneratorConstants.SORT_KEY + graphConfiguration.getTableName()); - return sortRowsStep; - } - - private GraphConfigurationInfo getGraphConfigInfoForFact( - CarbonDataLoadSchema carbonDataLoadSchema) throws GraphGeneratorException { - // - GraphConfigurationInfo graphConfiguration = new GraphConfigurationInfo(); - List dimensions = carbonDataLoadSchema.getCarbonTable() - .getDimensionByTableName(carbonDataLoadSchema.getCarbonTable().getFactTableName()); - prepareIsUseInvertedIndex(dimensions, graphConfiguration); - graphConfiguration - .setDimensions(CarbonSchemaParser.getTableDimensions(dimensions, carbonDataLoadSchema)); - graphConfiguration - .setActualDims(CarbonSchemaParser.getTableDimensions(dimensions, carbonDataLoadSchema)); - graphConfiguration - .setColumnPropertiesString(CarbonSchemaParser.getColumnPropertiesString(dimensions)); - graphConfiguration.setComplexTypeString(CarbonSchemaParser.getComplexTypeString(dimensions)); - prepareNoDictionaryMapping(dimensions, graphConfiguration); - graphConfiguration - .setColumnSchemaDetails(CarbonSchemaParser.getColumnSchemaDetails(dimensions)); - graphConfiguration.setTableOptionWrapper(getTableOptionWrapper()); - String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName(); - graphConfiguration.setTableName(factTableName); - StringBuilder dimString = new StringBuilder(); - // - int currentCount = - CarbonSchemaParser.getDimensionString(dimensions, dimString, 0, carbonDataLoadSchema); - StringBuilder noDictionarydimString = new StringBuilder(); - CarbonSchemaParser - .getNoDictionaryDimensionString(dimensions, noDictionarydimString, 0, carbonDataLoadSchema); - graphConfiguration.setNoDictionaryDims(noDictionarydimString.toString()); - - String tableString = - CarbonSchemaParser.getTableNameString(dimensions, carbonDataLoadSchema); - String dimensionColumnIds = CarbonSchemaParser.getColumnIdString(dimensions); - graphConfiguration.setDimensionTableNames(tableString); - graphConfiguration.setDimensionString(dimString.toString()); - graphConfiguration.setDimensionColumnIds(dimensionColumnIds); - graphConfiguration - .setForignKey(CarbonSchemaParser.getForeignKeyForTables(dimensions, carbonDataLoadSchema)); - List measures = carbonDataLoadSchema.getCarbonTable() - .getMeasureByTableName(carbonDataLoadSchema.getCarbonTable().getFactTableName()); - graphConfiguration - .setMeasuresString(CarbonSchemaParser.getMeasureString(measures, currentCount)); - graphConfiguration - .setHiersString(CarbonSchemaParser.getHierarchyString(dimensions, carbonDataLoadSchema)); - graphConfiguration.setHierColumnString( - CarbonSchemaParser.getHierarchyStringWithColumnNames(dimensions, carbonDataLoadSchema)); - graphConfiguration.setMeasureUniqueColumnNamesString( - CarbonSchemaParser.getMeasuresUniqueColumnNamesString(measures)); - graphConfiguration.setForeignKeyHierarchyString(CarbonSchemaParser - .getForeignKeyHierarchyString(dimensions, carbonDataLoadSchema, factTableName)); - graphConfiguration.setConnectionName("target"); - graphConfiguration.setHeirAndDimLens( - CarbonSchemaParser.getHeirAndCardinalityString(dimensions, carbonDataLoadSchema)); - //setting dimension store types - graphConfiguration.setColumnGroupsString(CarbonSchemaParser.getColumnGroups(dimensions)); - graphConfiguration.setPrimaryKeyString( - CarbonSchemaParser.getPrimaryKeyString(dimensions, carbonDataLoadSchema)); - graphConfiguration - .setDenormColumns(CarbonSchemaParser.getDenormColNames(dimensions, carbonDataLoadSchema)); - - graphConfiguration.setLevelAnddataType( - CarbonSchemaParser.getLevelAndDataTypeMapString(dimensions, carbonDataLoadSchema)); - - graphConfiguration.setForgienKeyAndPrimaryKeyMapString(CarbonSchemaParser - .getForeignKeyAndPrimaryKeyMapString(carbonDataLoadSchema.getDimensionRelationList())); - - graphConfiguration.setMdkeySize(CarbonSchemaParser.getMdkeySizeForFact(dimensions)); - Set measureColumn = new HashSet(measures.size()); - for (int i = 0; i < measures.size(); i++) { - measureColumn.add(measures.get(i).getColName()); - } - char[] type = new char[measureColumn.size()]; - Arrays.fill(type, 'n'); - graphConfiguration.setType(type); - graphConfiguration.setMeasureCount(measureColumn.size() + ""); - graphConfiguration.setHeirAndKeySizeString( - CarbonSchemaParser.getHeirAndKeySizeMapForFact(dimensions, carbonDataLoadSchema)); - graphConfiguration.setAggType(CarbonSchemaParser.getMeasuresAggragatorArray(measures)); - graphConfiguration.setMeasureNamesString(CarbonSchemaParser.getMeasuresNamesString(measures)); - graphConfiguration - .setActualDimensionColumns(CarbonSchemaParser.getActualDimensions(dimensions)); - graphConfiguration - .setDimensionColumnsDataType(CarbonSchemaParser.getDimensionsDataTypes(dimensions)); - //graphConfiguration.setNormHiers(CarbonSchemaParser.getNormHiers(table, schema)); - graphConfiguration.setMeasureDataTypeInfo(CarbonSchemaParser.getMeasuresDataType(measures)); - graphConfiguration.setStoreLocation( - this.databaseName + '/' + carbonDataLoadSchema.getCarbonTable().getFactTableName()); - graphConfiguration.setBlockletSize( - (instance.getProperty("com.huawei.unibi.carbon.blocklet.size", DEFAUL_BLOCKLET_SIZE))); - graphConfiguration.setMaxBlockletInFile( - (instance.getProperty("carbon.max.blocklet.in.file", DEFAULE_MAX_BLOCKLET_IN_FILE))); - graphConfiguration.setNumberOfCores( - (instance.getProperty(CarbonCommonConstants.NUM_CORES_LOADING, DEFAULT_NUMBER_CORES))); - - // check quotes required in query or Not - boolean isQuotesRequired = true; - String quote = CarbonSchemaParser.QUOTES; - graphConfiguration.setTableInputSqlQuery(CarbonSchemaParser - .getTableInputSQLQuery(dimensions, measures, - carbonDataLoadSchema.getCarbonTable().getFactTableName(), isQuotesRequired, - carbonDataLoadSchema)); - graphConfiguration - .setBatchSize((instance.getProperty("carbon.batch.size", DEFAULT_BATCH_SIZE))); - graphConfiguration.setSortSize((instance.getProperty("carbon.sort.size", DEFAULT_SORT_SIZE))); - graphConfiguration.setDimensionSqlQuery(CarbonSchemaParser - .getDimensionSQLQueries(dimensions, carbonDataLoadSchema, isQuotesRequired, quote)); - graphConfiguration.setMetaHeirString( - CarbonSchemaParser.getMetaHeirString(dimensions, carbonDataLoadSchema.getCarbonTable())); - graphConfiguration - .setDimCardinalities(CarbonSchemaParser.getCardinalities(dimensions, carbonDataLoadSchema)); - - graphConfiguration.setMeasures(CarbonSchemaParser.getMeasures(measures)); - graphConfiguration.setAGG(false); - return graphConfiguration; - } - - /** - * the method returns the table option wrapper - * - * @return - */ - private TableOptionWrapper getTableOptionWrapper() { - TableOptionWrapper tableOptionWrapper = TableOptionWrapper.getTableOptionWrapperInstance(); - tableOptionWrapper.setTableOption(schemaInfo.getSerializationNullFormat()); - tableOptionWrapper.setTableOption(schemaInfo.getBadRecordsLoggerEnable()); - tableOptionWrapper.setTableOption(schemaInfo.getBadRecordsLoggerAction()); - return tableOptionWrapper; - } - - public CarbonTable getTable() { - return carbonDataLoadSchema.getCarbonTable(); - } - - /** - * Preparing the boolean [] to map whether the dimension is no Dictionary or not. - * - * @param dims - * @param graphConfig - */ - private void prepareNoDictionaryMapping(List dims, - GraphConfigurationInfo graphConfig) { - List noDictionaryMapping = new ArrayList(); - for (CarbonDimension dimension : dims) { - // for complex type need to break the loop - if (dimension.getNumberOfChild() > 0) { - break; - } - - if (!dimension.getEncoder().contains(Encoding.DICTIONARY)) { - noDictionaryMapping.add(true); - //NoDictionaryMapping[index] = true; - } else { - noDictionaryMapping.add(false); - } - } - - graphConfig.setIsNoDictionaryDimMapping( - noDictionaryMapping.toArray(new Boolean[noDictionaryMapping.size()])); - } - - /** - * Preparing the boolean [] to map whether the dimension use inverted index or not. - * - * @param dims - * @param graphConfig - */ - private void prepareIsUseInvertedIndex(List dims, - GraphConfigurationInfo graphConfig) { - List isUseInvertedIndexList = new ArrayList(); - for (CarbonDimension dimension : dims) { - if (dimension.isUseInvertedIndex()) { - isUseInvertedIndexList.add(true); - } else { - isUseInvertedIndexList.add(false); - } - } - graphConfig.setIsUseInvertedIndex( - isUseInvertedIndexList.toArray(new Boolean[isUseInvertedIndexList.size()])); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java deleted file mode 100644 index 95a78230930..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * 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.graphgenerator; - -public final class GraphGeneratorConstants { - /** - * TABLE_INPUT - */ - public static final String TABLE_INPUT = "Table Input"; - /** - * CARBON_SURROGATE_KEY_GENERATOR - */ - public static final String CARBON_SURROGATE_KEY_GENERATOR = "Carbon Surrogate Key Generator"; - /** - * MDKEY_GENERATOR - */ - public static final String MDKEY_GENERATOR = "MDKey Generator"; - /** - * SORT_KEY - */ - public static final String SORT_KEY = "Sort keys"; - /** - * SORT_KEY - */ - public static final String SORT_KEY_AND_GROUPBY = "Sort keys And Group By Step"; - /** - * CARBON_SLICE_MERGER - */ - public static final String CARBON_SLICE_MERGER = "Carbon Slice Merger"; - /** - * SELECT_REQUIRED_VALUE - */ - public static final String SELECT_REQUIRED_VALUE = "Select Required Value"; - /** - * CSV Input - */ - public static final String CSV_INPUT = "CSV Input"; - /** - * CARBON_MDKEY_GENERATOR_ID - */ - public static final String MDKEY_GENERATOR_ID = "MDKeyGen"; - /** - * CARBON_SLICE_MERGER_ID - */ - public static final String CARBON_SLICE_MERGER_ID = "CarbonSliceMerger"; - /** - * CARBON_SLICE_MERGER_ID - */ - public static final String SORTKEY_ID = "SortKey"; - /** - * CARBON_CSV_BASED_SURROAGATEGEN_ID - */ - public static final String CARBON_CSV_BASED_SURROAGATEGEN_ID = "CarbonCSVBasedSurrogateGen"; - - private GraphGeneratorConstants() { - - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java deleted file mode 100644 index 6a3599434c3..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.graphgenerator; - -import java.util.Locale; - -public class GraphGeneratorException extends Exception { - - /** - * default serial version ID. - */ - private static final long serialVersionUID = 1L; - - /** - * The Error message. - */ - private String msg = ""; - - /** - * Constructor - * - * @param msg The error message for this exception. - */ - public GraphGeneratorException(String msg) { - super(msg); - this.msg = msg; - } - - /** - * Constructor - * - * @param msg The error message for this exception. - */ - public GraphGeneratorException(String msg, Throwable t) { - super(msg, t); - this.msg = msg; - } - - /** - * This method is used to get the localized message. - * - * @param locale - A Locale object represents a specific geographical, - * political, or cultural region. - * @return - Localized error message. - */ - public String getLocalizedMessage(Locale locale) { - return ""; - } - - /** - * getLocalizedMessage - */ - @Override public String getLocalizedMessage() { - return super.getLocalizedMessage(); - } - - /** - * getMessage - */ - public String getMessage() { - return this.msg; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java deleted file mode 100644 index f389a44c52a..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java +++ /dev/null @@ -1,894 +0,0 @@ -/* - * 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.graphgenerator.configuration; - -import java.util.Map; - -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper; -import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper; - -public class GraphConfigurationInfo { - private String connectionName; - - private String dbType; - - private String numberOfCores; - - private String storeLocation; - - private String tableName; - - private String blockletSize; - - private String maxBlockletInFile; - - private String batchSize; - - private Map dimCardinalities; - - private String[] dimensions; - - private String noDictionaryDims; - - private String[] measures; - - private String dimensionString; - - private String hiersString; - - private String measuresString; - - private String propertiesString; - - private String timeHeirString; - - private String metaHeirString; - - private String metaHeirQueryString; - - private String jndiName; - - private Map tableMeasuresAndDataTypeMap; - - private String tableInputSqlQuery; - - private String dimensionSqlQuery; - - private String sortSize; - - private boolean isAGG; - - private String driverclass; - - private String username; - - private String password; - - private String connectionUrl; - - private String[] actualDims; - - /** - * Sets the dimension:hirearchy#levelnames1,levelName2 - */ - private String dimensionTableNames; - - /** - * column Ids concatenated by a delimeter - */ - private String dimensionColumnIds; - - /** - * Agg type - */ - private String[] aggType; - - /** - * mdkeySize - */ - private String mdkeySize; - - /** - * complexTypeString - */ - private String complexTypeString; - - /** - * measureCount - */ - - private String measureCount; - - /** - * heirAndKeySizeString - */ - private String heirAndKeySizeString; - - /** - * hier and containing columns string - */ - private String hierColumnString; - - /** - * forignKey - */ - private String[] forignKey; - - /** - * Foreign Key and respective hierarchy - */ - private String foreignKeyHierarchyString; - - /** - * Primary key String - */ - private String primaryKeyString; - - /** - * Measure Names - */ - private String measureNamesString; - - /** - * Measure Names - */ - private String measureUniqueColumnNamesString; - - /** - * actualDimensionColumns - */ - private String actualDimensionColumns; - - /** - * normHiers - */ - private String normHiers; - - private String forgienKeyAndPrimaryKeyMapString; - - /** - * heirAndDimLens - */ - private String heirAndDimLens; - - /** - * measureTypeInfo - */ - private String measureDataTypeInfo; - - /** - * columnAndTableName_ColumnMapForAgg - */ - private String columnAndTableNameColumnMapForAgg; - /** - * denormColumns - */ - private String denormColumns; - private String[] aggClass; - /** - * type - */ - private char[] type; - private String levelAnddataType; - - private Boolean[] isNoDictionaryDimMapping; - - private Boolean[] isUseInvertedIndex; - - private String columnPropertiesString; - - /** - * wrapper object holding the columnschemadetails - */ - private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper; - - /** - * wrapper object holding the table options details needed while dataload - */ - private TableOptionWrapper tableOptionWrapper; - - /** - * It is column groups in below format - * 0,1~2~3,4,5,6~7~8,9 - * groups are - * ,-> all ordinal with different group id - * ~-> all ordinal with same group id - */ - private String columnGroupsString; - private String columnsDataTypeString; - /** - * @return isUseInvertedIndex - */ - public Boolean[] getIsUseInvertedIndex() { - return isUseInvertedIndex; - } - - /** - * @param isUseInvertedIndex the bool array whether use inverted index to set - */ - public void setIsUseInvertedIndex(Boolean[] isUseInvertedIndex) { - this.isUseInvertedIndex = isUseInvertedIndex; - } - - /** - * @param connectionName the connectionName to set - */ - public void setConnectionName(String connectionName) { - this.connectionName = connectionName; - } - - /** - * @return the numberOfCores - */ - public String getNumberOfCores() { - return numberOfCores; - } - - /** - * @param numberOfCores the numberOfCores to set - */ - public void setNumberOfCores(String numberOfCores) { - this.numberOfCores = numberOfCores; - } - - /** - * @param storeLocation the storeLocation to set - */ - public void setStoreLocation(String storeLocation) { - this.storeLocation = storeLocation; - } - - /** - * @return the tableName - */ - public String getTableName() { - return tableName; - } - - /** - * @param tableName the tableName to set - */ - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public String getComplexTypeString() { - return complexTypeString; - } - - public void setComplexTypeString(String complexTypeString) { - this.complexTypeString = complexTypeString; - } - - /** - * @param blockletSize the blockletSize to set - */ - public void setBlockletSize(String blockletSize) { - this.blockletSize = blockletSize; - } - - /** - * @param maxBlockletInFile the maxBlockletInFile to set - */ - public void setMaxBlockletInFile(String maxBlockletInFile) { - this.maxBlockletInFile = maxBlockletInFile; - } - - /** - * @return the batchSize - */ - public String getBatchSize() { - return batchSize; - } - - /** - * @param batchSize the batchSize to set - */ - public void setBatchSize(String batchSize) { - this.batchSize = batchSize; - } - - /** - * @return the dimCardinalities - */ - public Map getDimCardinalities() { - return dimCardinalities; - } - - /** - * @param dimCardinalities the dimCardinalities to set - */ - public void setDimCardinalities(Map dimCardinalities) { - this.dimCardinalities = dimCardinalities; - } - - /** - * @return the dimensions - */ - public String[] getDimensions() { - return dimensions; - } - - /** - * @param dimensions the dimensions to set - */ - public void setDimensions(String[] dimensions) { - this.dimensions = dimensions; - } - - /** - * @return the measures - */ - public String[] getMeasures() { - return measures; - } - - /** - * @param measures the measures to set - */ - public void setMeasures(String[] measures) { - this.measures = measures; - } - - /** - * @return the dimensionString - */ - public String getDimensionString() { - return dimensionString; - } - - /** - * @param dimensionString the dimensionString to set - */ - public void setDimensionString(String dimensionString) { - this.dimensionString = dimensionString; - } - - /** - * getNormHiers - * - * @return String - */ - public String getNormHiers() { - return normHiers; - } - - /** - * @return the hiersString - */ - public String getHiersString() { - return hiersString; - } - - /** - * @param hiersString the hiersString to set - */ - public void setHiersString(String hiersString) { - this.hiersString = hiersString; - } - - /** - * @return the measuresString - */ - public String getMeasuresString() { - return measuresString; - } - - /** - * @param measuresString the measuresString to set - */ - public void setMeasuresString(String measuresString) { - this.measuresString = measuresString; - } - - /** - * @return the propertiesString - */ - public String getPropertiesString() { - return propertiesString; - } - - /** - * @return the metaHeirString - */ - public String getMetaHeirString() { - return metaHeirString; - } - - /** - * @param metaHeirString the metaHeirString to set - */ - public void setMetaHeirString(String metaHeirString) { - this.metaHeirString = metaHeirString; - } - - /** - * @return the tableInputSqlQuery - */ - public String getTableInputSqlQuery() { - return tableInputSqlQuery; - } - - /** - * @param tableInputSqlQuery the tableInputSqlQuery to set - */ - public void setTableInputSqlQuery(String tableInputSqlQuery) { - this.tableInputSqlQuery = tableInputSqlQuery; - } - - /** - * @return the dimensionSqlQuery - */ - public String getDimensionSqlQuery() { - return dimensionSqlQuery; - } - - /** - * @param dimensionSqlQuery the dimensionSqlQuery to set - */ - public void setDimensionSqlQuery(String dimensionSqlQuery) { - this.dimensionSqlQuery = dimensionSqlQuery; - } - - /** - * @param sortSize the sortSize to set - */ - public void setSortSize(String sortSize) { - this.sortSize = sortSize; - } - - /** - * @return the isAGG - */ - public boolean isAGG() { - return isAGG; - } - - /** - * @param isAGG the isAGG to set - */ - public void setAGG(boolean isAGG) { - this.isAGG = isAGG; - } - - /** - * @return the driverclass - */ - public String getDriverclass() { - return driverclass; - } - - /** - * @param driverclass the driverclass to set - */ - public void setDriverclass(String driverclass) { - this.driverclass = driverclass; - } - - /** - * @return the username - */ - public String getUsername() { - return username; - } - - /** - * @param username the username to set - */ - public void setUsername(String username) { - this.username = username; - } - - /** - * @return the password - */ - public String getPassword() { - return password; - } - - /** - * @param password the password to set - */ - public void setPassword(String password) { - this.password = password; - } - - /** - * @return the connectionUrl - */ - public String getConnectionUrl() { - return connectionUrl; - } - - /** - * @param connectionUrl the connectionUrl to set - */ - public void setConnectionUrl(String connectionUrl) { - this.connectionUrl = connectionUrl; - } - - /** - * @return the actualDims - */ - public String[] getActualDims() { - return actualDims; - } - - /** - * @param actualDims the actualDims to set - */ - public void setActualDims(String[] actualDims) { - this.actualDims = actualDims; - } - - /** - * @return the dimensionTableNames - */ - public String getDimensionTableNames() { - return dimensionTableNames; - } - - /** - * @param dimensionTableNames the dimensionTableNames to set - */ - public void setDimensionTableNames(String dimensionTableNames) { - this.dimensionTableNames = dimensionTableNames; - } - - /** - * @return - */ - public String getDimensionColumnIds() { - return dimensionColumnIds; - } - - /** - * @param dimensionColumnIds column Ids for dimensions in a table - */ - public void setDimensionColumnIds(String dimensionColumnIds) { - this.dimensionColumnIds = dimensionColumnIds; - } - - /** - * getMdkeySize - * - * @return String - */ - public String getMdkeySize() { - return mdkeySize; - } - - /** - * setMdkeySize - * - * @param mdkeySize void - */ - public void setMdkeySize(String mdkeySize) { - this.mdkeySize = mdkeySize; - } - - /** - * getMeasureCount - * - * @return String - */ - public String getMeasureCount() { - return measureCount; - } - - /** - * setMeasureCount - * - * @param measureCount void - */ - public void setMeasureCount(String measureCount) { - this.measureCount = measureCount; - } - - /** - * getHeirAndKeySizeString - * - * @return String - */ - public String getHeirAndKeySizeString() { - return heirAndKeySizeString; - } - - /** - * setHeirAndKeySizeString - * - * @param heirAndKeySizeString void - */ - public void setHeirAndKeySizeString(String heirAndKeySizeString) { - this.heirAndKeySizeString = heirAndKeySizeString; - } - - /** - * @return Returns the hierColumnString. - */ - public String getHierColumnString() { - return hierColumnString; - } - - /** - * @param hierColumnString The hierColumnString to set. - */ - public void setHierColumnString(String hierColumnString) { - this.hierColumnString = hierColumnString; - } - - /** - * @param forignKey The forignKey to set. - */ - public void setForignKey(String[] forignKey) { - this.forignKey = forignKey; - } - - /** - * @return Returns the foreignKeyHierarchyString. - */ - public String getForeignKeyHierarchyString() { - return foreignKeyHierarchyString; - } - - /** - * @param foreignKeyHierarchyString The foreignKeyHierarchyString to set. - */ - public void setForeignKeyHierarchyString(String foreignKeyHierarchyString) { - this.foreignKeyHierarchyString = foreignKeyHierarchyString; - } - - /** - * @return Returns the primaryKeyString. - */ - public String getPrimaryKeyString() { - return primaryKeyString; - } - - /** - * @param primaryKeyString The primaryKeyString to set. - */ - public void setPrimaryKeyString(String primaryKeyString) { - this.primaryKeyString = primaryKeyString; - } - - /** - * @return the measureNamesString - */ - public String getMeasureNamesString() { - return measureNamesString; - } - - /** - * @param measureNamesString the measureNamesString to set - */ - public void setMeasureNamesString(String measureNamesString) { - this.measureNamesString = measureNamesString; - } - - /** - * @return Returns the aggType. - */ - public String[] getAggType() { - return aggType; - } - - /** - * @param aggType The aggType to set. - */ - public void setAggType(String[] aggType) { - this.aggType = aggType; - } - - /** - * @return Returns the actualDimensionColumns. - */ - public String getActualDimensionColumns() { - return actualDimensionColumns; - } - - /** - * @param actualDimensionColumns The actualDimensionColumns to set. - */ - public void setActualDimensionColumns(String actualDimensionColumns) { - this.actualDimensionColumns = actualDimensionColumns; - } - - /** - * getForgienKeyAndPrimaryKeyMapString - * - * @return String - */ - public String getForgienKeyAndPrimaryKeyMapString() { - return forgienKeyAndPrimaryKeyMapString; - } - - /** - * setForgienKeyAndPrimaryKeyMapString - * - * @param forgienKeyAndPrimaryKeyMapString void - */ - public void setForgienKeyAndPrimaryKeyMapString(String forgienKeyAndPrimaryKeyMapString) { - this.forgienKeyAndPrimaryKeyMapString = forgienKeyAndPrimaryKeyMapString; - } - - /** - * @return Returns the heirAndDimLens. - */ - public String getHeirAndDimLens() { - return heirAndDimLens; - } - - /** - * @param heirAndDimLens The heirAndDimLens to set. - */ - public void setHeirAndDimLens(String heirAndDimLens) { - this.heirAndDimLens = heirAndDimLens; - } - - /** - * @return Returns the measureDataTypeInfo. - */ - public String getMeasureDataTypeInfo() { - return measureDataTypeInfo; - } - - /** - * @param measureDataTypeInfo The measureDataTypeInfo to set. - */ - public void setMeasureDataTypeInfo(String measureDataTypeInfo) { - this.measureDataTypeInfo = measureDataTypeInfo; - } - - public String getColumnAndTableNameColumnMapForAgg() { - return columnAndTableNameColumnMapForAgg; - } - - /** - * @return Returns the denormColumns. - */ - public String getDenormColumns() { - return denormColumns; - } - - /** - * @param denormColumns The denormColumns to set. - */ - public void setDenormColumns(String denormColumns) { - this.denormColumns = denormColumns; - } - - /** - * @return the aggClass - */ - public String[] getAggClass() { - return aggClass; - } - - /** - * @param measureUniqueColumnNamesString the measureUniqueColumnNamesString to set - */ - public void setMeasureUniqueColumnNamesString(String measureUniqueColumnNamesString) { - this.measureUniqueColumnNamesString = measureUniqueColumnNamesString; - } - - /** - * @param type the type to set - */ - public void setType(char[] type) { - this.type = type; - } - - public String getLevelAnddataType() { - return levelAnddataType; - } - - public void setLevelAnddataType(String levelAnddataType) { - this.levelAnddataType = levelAnddataType; - } - - /** - * getNoDictionaryDims. - * - * @return - */ - public String getNoDictionaryDims() { - return noDictionaryDims; - } - - /** - * setNoDictionaryDims. - * - * @param noDictionaryDims - */ - public void setNoDictionaryDims(String noDictionaryDims) { - this.noDictionaryDims = noDictionaryDims; - } - - /** - * Set Wrapper Object having the columnschemadetails - * - * @param columnSchemaDetailsWrapper - */ - public void setColumnSchemaDetails(ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper) { - this.columnSchemaDetailsWrapper = columnSchemaDetailsWrapper; - } - - /** - * return the Wrapper Object having the columnschemadetails - * @return - */ - public ColumnSchemaDetailsWrapper getColumnSchemaDetails() { - return columnSchemaDetailsWrapper; - } - - /** - * set wraper object having table options needed while dataload - * - * @return - */ - public void setTableOptionWrapper(TableOptionWrapper tableOptionWrapper) { - this.tableOptionWrapper = tableOptionWrapper; - } - - /** - * method returns the table options detail wrapper instance. - * @return - */ - public TableOptionWrapper getTableOptionWrapper() { - return tableOptionWrapper; - } - - public void setColumnGroupsString(String columnGroups) { - this.columnGroupsString = columnGroups; - } - - /** - * @return columngroups - */ - public String getColumnGroupsString() { - return columnGroupsString; - } - - public Boolean[] getIsNoDictionaryDimMapping() { - return isNoDictionaryDimMapping; - } - - public void setIsNoDictionaryDimMapping(Boolean[] isNoDictionaryDimMapping) { - this.isNoDictionaryDimMapping = isNoDictionaryDimMapping; - } - - public void setColumnPropertiesString(String columnPropertiesString) { - this.columnPropertiesString = columnPropertiesString; - } - - public String getColumnPropertiesString() { - return this.columnPropertiesString; - } - - /** - * @return columngroups - */ - public String getDimensionColumnsDataType() { - return columnsDataTypeString; - } - - public void setDimensionColumnsDataType(String columnsDataTypeString) { - this.columnsDataTypeString = columnsDataTypeString; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java deleted file mode 100644 index 2f926991d48..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java +++ /dev/null @@ -1,531 +0,0 @@ -/* - * 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.mdkeygen; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.common.logging.impl.StandardLogService; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.block.SegmentProperties; -import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.metadata.CarbonMetadata; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; -import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory; -import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.DataTypeUtil; -import org.apache.carbondata.core.util.path.CarbonStorePath; -import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.mdkeygen.file.FileData; -import org.apache.carbondata.processing.mdkeygen.file.FileManager; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; -import org.apache.carbondata.processing.store.CarbonDataFileAttributes; -import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar; -import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel; -import org.apache.carbondata.processing.store.CarbonFactHandler; -import org.apache.carbondata.processing.store.SingleThreadFinalSortFilesMerger; -import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; -import org.apache.carbondata.processing.util.NonDictionaryUtil; - -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.row.RowMeta; -import org.pentaho.di.core.row.ValueMeta; -import org.pentaho.di.core.row.ValueMetaInterface; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStep; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; - -public class MDKeyGenStep extends BaseStep { - private static final LogService LOGGER = - LogServiceFactory.getLogService(MDKeyGenStep.class.getName()); - - /** - * carbon mdkey generator step data class - */ - private MDKeyGenStepData data; - - /** - * carbon mdkey generator step meta - */ - private MDKeyGenStepMeta meta; - - /** - * dimension length - */ - private int dimensionCount; - - /** - * table name - */ - private String tableName; - - /** - * table blocksize in MB - */ - private int blockSize; - - /** - * File manager - */ - private IFileManagerComposite fileManager; - - private Map complexIndexMap; - - private SegmentProperties segmentProperties; - - private int[] colCardinality; - - private List wrapperColumnSchema; - - /** - * readCounter - */ - private long readCounter; - - /** - * writeCounter - */ - private long writeCounter; - - private int measureCount; - - private String dataFolderLocation; - - private SingleThreadFinalSortFilesMerger finalMerger; - - /** - * dataHandler - */ - private CarbonFactHandler dataHandler; - - private char[] aggType; - - private String storeLocation; - - private int[] dimLens; - - /** - * to check whether dimension is of dictionary type - * or not - */ - private boolean[] isNoDictionaryDimension; - - /** - * to check whether dimension use inverted index - * or not - */ - private boolean[] isUseInvertedIndex; - - /** - * CarbonMDKeyGenStep - * - * @param stepMeta - * @param stepDataInterface - * @param copyNr - * @param transMeta - * @param trans - */ - public MDKeyGenStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - super(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - /** - * Perform the equivalent of processing one row. Typically this means - * reading a row from input (getRow()) and passing a row to output - * (putRow)). - * - * @param smi The steps metadata to work with - * @param sdi The steps temporary working data to work with (database - * connections, result sets, caches, temporary variables, etc.) - * @return false if no more rows can be processed or an error occurred. - * @throws KettleException - */ - public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException { - meta = (MDKeyGenStepMeta) smi; - StandardLogService.setThreadName(meta.getPartitionID(), null); - data = (MDKeyGenStepData) sdi; - - meta.initialize(); - Object[] row = getRow(); - if (first) { - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordMdkGenerateTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - first = false; - - data.outputRowMeta = new RowMeta(); - boolean isExecutionRequired = setStepConfiguration(); - - if (!isExecutionRequired) { - processingComplete(); - return false; - } - setStepOutputInterface(); - } - - if (null != row) { - putRow(data.outputRowMeta, new Object[measureCount + 1]); - return true; - } - - try { - initDataHandler(); - dataHandler.initialise(); - finalMerger.startFinalMerge(); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValue2MdkAdd2FileTime( - meta.getPartitionID(), System.currentTimeMillis()); - while (finalMerger.hasNext()) { - Object[] r = finalMerger.next(); - readCounter++; - Object[] outputRow = process(r); - dataHandler.addDataToStore(outputRow); - writeCounter++; - } - } catch (CarbonDataWriterException e) { - LOGGER.error(e, "Failed for table: " + this.tableName + " in MDKeyGenStep"); - throw new KettleException("Error while initializing data handler : " + e.getMessage()); - } catch (Exception e) { - LOGGER.error(e, "Failed for table: " + this.tableName + " in MDKeyGenStep"); - throw new KettleException("There is an unexpected error: " + e.getMessage()); - } finally { - try { - dataHandler.finish(); - } catch (CarbonDataWriterException e) { - LOGGER.error(e, "Failed for table: " + this.tableName + " in finishing data handler"); - } catch (Exception e) { - LOGGER.error(e, "Failed for table: " + this.tableName + " in finishing data handler"); - } - } - LOGGER.info("Record Procerssed For table: " + this.tableName); - String logMessage = - "Finished Carbon Mdkey Generation Step: Read: " + readCounter + ": Write: " + writeCounter; - LOGGER.info(logMessage); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(writeCounter); - processingComplete(); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValue2MdkAdd2FileTime( - meta.getPartitionID(), System.currentTimeMillis()); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordMdkGenerateTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - return false; - } - - private void processingComplete() throws KettleException { - if (null != dataHandler) { - try { - dataHandler.closeHandler(); - } catch (CarbonDataWriterException e) { - LOGGER.error(e, e.getMessage()); - throw new KettleException(e.getMessage()); - } catch (Exception e) { - LOGGER.error(e, e.getMessage()); - throw new KettleException("There is an unexpected error: " + e.getMessage()); - } - } - setOutputDone(); - } - - /** - * This method will be used to get and update the step properties which will - * required to run this step - */ - private boolean setStepConfiguration() { - this.tableName = meta.getTableName(); - storeLocation = CarbonDataProcessorUtil - .getLocalDataFolderLocation(meta.getDatabaseName(), meta.getTableName(), - String.valueOf(meta.getTaskNo()), meta.getPartitionID(), meta.getSegmentId() + "", - false); - isNoDictionaryDimension = - NonDictionaryUtil.convertStringToBooleanArr(meta.getNoDictionaryDimsMapping()); - isUseInvertedIndex = - NonDictionaryUtil.convertStringToBooleanArr(meta.getIsUseInvertedIndex()); - fileManager = new FileManager(); - fileManager.setName(CarbonCommonConstants.LOAD_FOLDER + meta.getSegmentId() - + CarbonCommonConstants.FILE_INPROGRESS_STATUS); - - if (!(new File(storeLocation).exists())) { - LOGGER.error("Local data load folder location does not exist: " + storeLocation); - return false; - } - - this.meta.setNoDictionaryCount( - NonDictionaryUtil.extractNoDictionaryCount(this.meta.getNoDictionaryDims())); - - String levelCardinalityFilePath = storeLocation + File.separator + - CarbonCommonConstants.LEVEL_METADATA_FILE + meta.getTableName() - + CarbonCommonConstants.CARBON_METADATA_EXTENSION; - int[] dimLensWithComplex = null; - try { - dimLensWithComplex = CarbonUtil.getCardinalityFromLevelMetadataFile(levelCardinalityFilePath); - } catch (IOException e) { - LOGGER.error("Level cardinality file :: " + e.getMessage()); - return false; - } - if (null == dimLensWithComplex) { - return false; - } - List dimsLenList = new ArrayList(); - for (int eachDimLen : dimLensWithComplex) { - if (eachDimLen != 0) dimsLenList.add(eachDimLen); - } - dimLens = new int[dimsLenList.size()]; - for (int i = 0; i < dimsLenList.size(); i++) { - dimLens[i] = dimsLenList.get(i); - } - - this.dimensionCount = meta.getDimensionCount(); - - int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount(); - int[] simpleDimsLen = new int[simpleDimsCount]; - System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount); - - CarbonTable carbonTable = CarbonMetadata.getInstance() - .getCarbonTable(meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + tableName); - wrapperColumnSchema = CarbonUtil - .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName), - carbonTable.getMeasureByTableName(tableName)); - blockSize = carbonTable.getBlockSizeInMB(); - colCardinality = - CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchema); - segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality); - // Actual primitive dimension used to generate start & end key - - data.generator = segmentProperties.getDimensionKeyGenerator(); - - //To Set MDKey Index of each primitive type in complex type - int surrIndex = simpleDimsCount; - Iterator> complexMap = - meta.getComplexTypes().entrySet().iterator(); - complexIndexMap = new HashMap(meta.getComplexDimsCount()); - while (complexMap.hasNext()) { - Entry complexDataType = complexMap.next(); - complexDataType.getValue().setOutputArrayIndex(0); - complexIndexMap.put(simpleDimsCount, complexDataType.getValue()); - simpleDimsCount++; - List primitiveTypes = new ArrayList(); - complexDataType.getValue().getAllPrimitiveChildren(primitiveTypes); - for (GenericDataType eachPrimitive : primitiveTypes) { - eachPrimitive.setSurrogateIndex(surrIndex++); - } - } - - this.measureCount = meta.getMeasureCount(); - - String metaDataFileName = CarbonCommonConstants.MEASURE_METADATA_FILE_NAME + this.tableName - + CarbonCommonConstants.MEASUREMETADATA_FILE_EXT - + CarbonCommonConstants.FILE_INPROGRESS_STATUS; - - FileData fileData = new FileData(metaDataFileName, storeLocation); - fileManager.add(fileData); - // Set the data file location - this.dataFolderLocation = - storeLocation + File.separator + CarbonCommonConstants.SORT_TEMP_FILE_LOCATION; - return true; - } - - private void initDataHandler() { - int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount(); - int[] simpleDimsLen = new int[simpleDimsCount]; - System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount); - CarbonDataFileAttributes carbonDataFileAttributes = - new CarbonDataFileAttributes(meta.getTaskNo(), meta.getFactTimeStamp()); - initAggType(); - String carbonDataDirectoryPath = getCarbonDataFolderLocation(); - finalMerger = new SingleThreadFinalSortFilesMerger(dataFolderLocation, tableName, - dimensionCount - meta.getComplexDimsCount(), meta.getComplexDimsCount(), measureCount, - meta.getNoDictionaryCount(), aggType, isNoDictionaryDimension, true); - CarbonFactDataHandlerModel carbonFactDataHandlerModel = getCarbonFactDataHandlerModel(); - carbonFactDataHandlerModel.setPrimitiveDimLens(simpleDimsLen); - carbonFactDataHandlerModel.setCarbonDataFileAttributes(carbonDataFileAttributes); - carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath); - carbonFactDataHandlerModel.setIsUseInvertedIndex(isUseInvertedIndex); - if (meta.getNoDictionaryCount() > 0 || meta.getComplexDimsCount() > 0) { - carbonFactDataHandlerModel.setMdKeyIndex(measureCount + 1); - } else { - carbonFactDataHandlerModel.setMdKeyIndex(measureCount); - } - dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel); - } - - /** - * This method will create a model object for carbon fact data handler - * - * @return - */ - private CarbonFactDataHandlerModel getCarbonFactDataHandlerModel() { - CarbonFactDataHandlerModel carbonFactDataHandlerModel = new CarbonFactDataHandlerModel(); - carbonFactDataHandlerModel.setDatabaseName(meta.getDatabaseName()); - carbonFactDataHandlerModel.setTableName(tableName); - carbonFactDataHandlerModel.setMeasureCount(measureCount); - carbonFactDataHandlerModel.setMdKeyLength(data.generator.getKeySizeInBytes()); - carbonFactDataHandlerModel.setStoreLocation(storeLocation); - carbonFactDataHandlerModel.setDimLens(dimLens); - carbonFactDataHandlerModel.setNoDictionaryCount(meta.getNoDictionaryCount()); - carbonFactDataHandlerModel.setDimensionCount(dimensionCount); - carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap); - carbonFactDataHandlerModel.setSegmentProperties(segmentProperties); - carbonFactDataHandlerModel.setColCardinality(colCardinality); - carbonFactDataHandlerModel.setDataWritingRequest(true); - carbonFactDataHandlerModel.setAggType(aggType); - carbonFactDataHandlerModel.setFactDimLens(dimLens); - carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema); - carbonFactDataHandlerModel.setBlockSizeInMB(blockSize); - return carbonFactDataHandlerModel; - } - - private void initAggType() { - aggType = new char[measureCount]; - Arrays.fill(aggType, 'n'); - CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable( - meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName()); - List measures = carbonTable.getMeasureByTableName(meta.getTableName()); - for (int i = 0; i < measureCount; i++) { - aggType[i] = DataTypeUtil.getAggType(measures.get(i).getDataType()); - } - } - - /** - * This method will be used for setting the output interface. - * Output interface is how this step will process the row to next step - */ - private void setStepOutputInterface() { - ValueMetaInterface[] out = new ValueMetaInterface[measureCount + 1]; - - for (int i = 0; i < measureCount; i++) { - out[i] = new ValueMeta("measure" + i, ValueMetaInterface.TYPE_NUMBER, - ValueMetaInterface.STORAGE_TYPE_NORMAL); - out[i].setStorageMetadata(new ValueMeta("measure" + i, ValueMetaInterface.TYPE_NUMBER, - ValueMetaInterface.STORAGE_TYPE_NORMAL)); - } - - out[out.length - 1] = new ValueMeta("id", ValueMetaInterface.TYPE_BINARY, - ValueMetaInterface.STORAGE_TYPE_BINARY_STRING); - out[out.length - 1].setStorageMetadata(new ValueMeta("id", ValueMetaInterface.TYPE_STRING, - ValueMetaInterface.STORAGE_TYPE_NORMAL)); - out[out.length - 1].setLength(256); - out[out.length - 1].setStringEncoding(CarbonCommonConstants.BYTE_ENCODING); - out[out.length - 1].getStorageMetadata().setStringEncoding(CarbonCommonConstants.BYTE_ENCODING); - - data.outputRowMeta.setValueMetaList(Arrays.asList(out)); - } - - /** - * This method will be used to get the row from previous step and then it - * will generate the mdkey and then send the mdkey to next step - * - * @param row input row - * @throws KettleException - */ - private Object[] process(Object[] row) throws KettleException { - Object[] outputRow = null; - // adding one for the high cardinality dims byte array. - if (meta.getNoDictionaryCount() > 0 || meta.getComplexDimsCount() > 0) { - outputRow = new Object[measureCount + 1 + 1]; - } else { - outputRow = new Object[measureCount + 1]; - } - - int l = 0; - int index = 0; - for (int i = 0; i < measureCount; i++) { - if (aggType[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - outputRow[l++] = NonDictionaryUtil.getMeasure(index++, row); - } else if (aggType[i] == CarbonCommonConstants.BIG_INT_MEASURE) { - outputRow[l++] = (Long) NonDictionaryUtil.getMeasure(index++, row); - } else { - outputRow[l++] = (Double) NonDictionaryUtil.getMeasure(index++, row); - } - } - outputRow[l] = NonDictionaryUtil.getByteArrayForNoDictionaryCols(row); - - int[] highCardExcludedRows = new int[segmentProperties.getDimColumnsCardinality().length]; - for (int i = 0; i < highCardExcludedRows.length; i++) { - Object key = NonDictionaryUtil.getDimension(i, row); - highCardExcludedRows[i] = (Integer) key; - } - try { - outputRow[outputRow.length - 1] = data.generator.generateKey(highCardExcludedRows); - } catch (KeyGenException e) { - throw new KettleException("unable to generate the mdkey", e); - } - - return outputRow; - } - - /** - * This method will get the store location for the given path, segment id and partition id - * - * @return data directory path - */ - private String getCarbonDataFolderLocation() { - String carbonStorePath = - CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS); - CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable( - meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName()); - CarbonTablePath carbonTablePath = - CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTable.getCarbonTableIdentifier()); - String carbonDataDirectoryPath = - carbonTablePath.getCarbonDataDirectoryPath(meta.getPartitionID(), meta.getSegmentId() + ""); - return carbonDataDirectoryPath; - } - - /** - * Initialize and do work where other steps need to wait for... - * - * @param smi The metadata to work with - * @param sdi The data to initialize - * @return step initialize or not - */ - public boolean init(StepMetaInterface smi, StepDataInterface sdi) { - meta = (MDKeyGenStepMeta) smi; - data = (MDKeyGenStepData) sdi; - - return super.init(smi, sdi); - } - - /** - * Dispose of this step: close files, empty logs, etc. - * - * @param smi The metadata to work with - * @param sdi The data to dispose of - */ - public void dispose(StepMetaInterface smi, StepDataInterface sdi) { - meta = (MDKeyGenStepMeta) smi; - data = (MDKeyGenStepData) sdi; - super.dispose(smi, sdi); - dataHandler = null; - finalMerger = null; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java deleted file mode 100644 index b2c682c2e60..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.mdkeygen; - -import org.apache.carbondata.core.keygenerator.KeyGenerator; - -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.step.BaseStepData; -import org.pentaho.di.trans.step.StepDataInterface; - -public class MDKeyGenStepData extends BaseStepData implements StepDataInterface { - /** - * outputRowMeta - */ - protected RowMetaInterface outputRowMeta; - - /** - * generator for each column independently - */ - protected KeyGenerator generator; - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java deleted file mode 100644 index c1be2ef0e4d..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java +++ /dev/null @@ -1,529 +0,0 @@ -/* - * 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.mdkeygen; - -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.processing.datatypes.ArrayDataType; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.datatypes.PrimitiveDataType; -import org.apache.carbondata.processing.datatypes.StructDataType; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; - -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Counter; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.exception.KettleXMLException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.xml.XMLHandler; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.repository.ObjectId; -import org.pentaho.di.repository.Repository; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStepMeta; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.w3c.dom.Node; - -public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface { - /** - * for i18n purposes - */ - private static Class pkg = MDKeyGenStepMeta.class; - - /** - * tableName - */ - private String tableName; - - /** - * numberOfCores - */ - private String numberOfCores; - - /** - * databaseName - */ - private String databaseName; - - /** - * aggregateLevels - */ - private String aggregateLevels; - - /** - * measureCount - */ - private String measureCount; - - /** - * dimensionCount - */ - private String dimensionCount; - - /** - * complexDimsCount - */ - private String complexDimsCount; - - /** - * ComplexTypeString - */ - private String complexTypeString; - - private Map complexTypes; - - /** - * It is column groups in below format - * 0,1~2~3,4,5,6~7~8,9 - * groups are - * ,-> all ordinal with different group id - * ~-> all ordinal with same group id - */ - private String columnGroupsString; - private String noDictionaryDims; - - /** - * noDictionaryCount - */ - private int noDictionaryCount; - - private String measureDataType; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - /** - * new load start time - */ - private String factTimeStamp; - /** - * partitionID - */ - private String partitionID; - /** - * Id of the load folder - */ - private String segmentId; - /** - * To determine the column whether is dictionary or not. - */ - private String noDictionaryDimsMapping; - /** - * To determine the column whether use inverted index or not. - */ - private String isUseInvertedIndex; - - /** - * Constructor - */ - public MDKeyGenStepMeta() { - super(); - } - - @Override public void setDefault() { - tableName = ""; - numberOfCores = ""; - aggregateLevels = ""; - tableName = ""; - databaseName = ""; - columnGroupsString = ""; - noDictionaryDims = ""; - measureDataType = ""; - taskNo = ""; - factTimeStamp = ""; - partitionID = ""; - segmentId = ""; - noDictionaryDimsMapping = ""; - } - - public String getXML() { - StringBuilder retval = new StringBuilder(150); - - retval.append(" ").append(XMLHandler.addTagValue("TableName", tableName)); - retval.append(" ").append(XMLHandler.addTagValue("AggregateLevels", aggregateLevels)); - retval.append(" ").append(XMLHandler.addTagValue("NumberOfCores", numberOfCores)); - retval.append(" ").append(XMLHandler.addTagValue("tableName", tableName)); - retval.append(" ").append(XMLHandler.addTagValue("databaseName", databaseName)); - retval.append(" ").append(XMLHandler.addTagValue("noDictionaryDims", noDictionaryDims)); - retval.append(" ").append(XMLHandler.addTagValue("measureCount", measureCount)); - retval.append(" ").append(XMLHandler.addTagValue("dimensionsStoreType", columnGroupsString)); - retval.append(" ").append(XMLHandler.addTagValue("dimensionCount", dimensionCount)); - retval.append(" ").append(XMLHandler.addTagValue("complexDimsCount", complexDimsCount)); - retval.append(" ").append(XMLHandler.addTagValue("complexTypeString", complexTypeString)); - retval.append(" ").append(XMLHandler.addTagValue("measureDataType", measureDataType)); - retval.append(" ").append(XMLHandler.addTagValue("taskNo", taskNo)); - retval.append(" ").append(XMLHandler.addTagValue("factTimeStamp", factTimeStamp)); - retval.append(" ").append(XMLHandler.addTagValue("factTimeStamp", factTimeStamp)); - retval.append(" ").append(XMLHandler.addTagValue("partitionID", partitionID)); - retval.append(" ").append(XMLHandler.addTagValue("isUseInvertedIndex", isUseInvertedIndex)); - retval.append(" ").append(XMLHandler.addTagValue("segmentId", segmentId)); - retval.append(" ") - .append(XMLHandler.addTagValue("noDictionaryDimsMapping", noDictionaryDimsMapping)); - return retval.toString(); - } - - @Override - public void loadXML(Node stepnode, List databases, Map counters) - throws KettleXMLException { - try { - tableName = XMLHandler.getTagValue(stepnode, "TableName"); - aggregateLevels = XMLHandler.getTagValue(stepnode, "AggregateLevels"); - numberOfCores = XMLHandler.getTagValue(stepnode, "NumberOfCores"); - databaseName = XMLHandler.getTagValue(stepnode, "databaseName"); - tableName = XMLHandler.getTagValue(stepnode, "tableName"); - noDictionaryDims = XMLHandler.getTagValue(stepnode, "noDictionaryDims"); - measureCount = XMLHandler.getTagValue(stepnode, "measureCount"); - columnGroupsString = XMLHandler.getTagValue(stepnode, "dimensionsStoreType"); - dimensionCount = XMLHandler.getTagValue(stepnode, "dimensionCount"); - complexDimsCount = XMLHandler.getTagValue(stepnode, "complexDimsCount"); - complexTypeString = XMLHandler.getTagValue(stepnode, "complexTypeString"); - measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType"); - taskNo = XMLHandler.getTagValue(stepnode, "taskNo"); - factTimeStamp = XMLHandler.getTagValue(stepnode, "factTimeStamp"); - partitionID = XMLHandler.getTagValue(stepnode, "partitionID"); - isUseInvertedIndex = XMLHandler.getTagValue(stepnode, "isUseInvertedIndex"); - segmentId = XMLHandler.getTagValue(stepnode, "segmentId"); - noDictionaryDimsMapping = XMLHandler.getTagValue(stepnode, "noDictionaryDimsMapping"); - } catch (Exception e) { - throw new KettleXMLException("Unable to read step info from XML node", e); - } - } - - @Override public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep) - throws KettleException { - try { - rep.saveStepAttribute(idTransformation, idStep, "TableName", tableName); - rep.saveStepAttribute(idTransformation, idStep, "AggregateLevels", aggregateLevels); - rep.saveStepAttribute(idTransformation, idStep, "NumberOfCores", numberOfCores); - rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName); - rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName); - rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDims", noDictionaryDims); - rep.saveStepAttribute(idTransformation, idStep, "measureCount", measureCount); - rep.saveStepAttribute(idTransformation, idStep, "dimensionsStoreType", columnGroupsString); - rep.saveStepAttribute(idTransformation, idStep, "dimensionCount", dimensionCount); - rep.saveStepAttribute(idTransformation, idStep, "complexDimsCount", complexDimsCount); - rep.saveStepAttribute(idTransformation, idStep, "complexTypeString", complexTypeString); - rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType); - rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo); - rep.saveStepAttribute(idTransformation, idStep, "factTimeStamp", factTimeStamp); - rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID); - rep.saveStepAttribute(idTransformation, idStep, "isUseInvertedIndex", isUseInvertedIndex); - rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId); - rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDimsMapping", - noDictionaryDimsMapping); - } catch (Exception e) { - throw new KettleException( - BaseMessages.getString(pkg, "TemplateStep.Exception.UnableToSaveStepInfoToRepository") - + idStep, e); - } - - } - - @Override public void readRep(Repository rep, ObjectId idStep, List databases, - Map counters) throws KettleException { - try { - tableName = rep.getStepAttributeString(idStep, "TableName"); - aggregateLevels = rep.getStepAttributeString(idStep, "AggregateLevels"); - numberOfCores = rep.getStepAttributeString(idStep, "NumberOfCores"); - databaseName = rep.getStepAttributeString(idStep, "databaseName"); - tableName = rep.getStepAttributeString(idStep, "tableName"); - noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDims"); - measureCount = rep.getStepAttributeString(idStep, "measureCount"); - columnGroupsString = rep.getStepAttributeString(idStep, "dimensionsStoreType"); - dimensionCount = rep.getStepAttributeString(idStep, "dimensionCount"); - complexDimsCount = rep.getStepAttributeString(idStep, "complexDimsCount"); - complexTypeString = rep.getStepAttributeString(idStep, "complexTypeString"); - measureDataType = rep.getStepAttributeString(idStep, "measureDataType"); - taskNo = rep.getStepAttributeString(idStep, "taskNo"); - factTimeStamp = rep.getStepAttributeString(idStep, "factTimeStamp"); - partitionID = rep.getStepAttributeString(idStep, "partitionID"); - isUseInvertedIndex = rep.getStepAttributeString(idStep, "isUseInvertedIndex"); - segmentId = rep.getStepAttributeString(idStep, "segmentId"); - noDictionaryDimsMapping = rep.getStepAttributeString(idStep, "noDictionaryDimsMapping"); - } catch (Exception e) { - throw new KettleException(BaseMessages - .getString(pkg, "CarbonMDKeyStepMeta.Exception.UnexpectedErrorInReadingStepInfo"), e); - } - } - - @Override - public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - return new MDKeyGenStep(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - @Override - public void check(List remarks, TransMeta transMeta, StepMeta stepMeta, - RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) { - CarbonDataProcessorUtil.checkResult(remarks, stepMeta, input); - } - - @Override public StepDataInterface getStepData() { - return new MDKeyGenStepData(); - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void setAggregateLevels(String aggregateLevels) { - this.aggregateLevels = aggregateLevels; - } - - public Map getComplexTypes() { - return complexTypes; - } - - public void setNumberOfCores(String numberOfCores) { - this.numberOfCores = numberOfCores; - } - - /** - * @return the databaseName - */ - public String getDatabaseName() { - return databaseName; - } - - /** - * @param databaseName the databaseName to set - */ - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - /** - * @return the measureCount - */ - public int getMeasureCount() { - return Integer.parseInt(measureCount); - } - - /** - * @param measureCount the measureCount to set - */ - public void setMeasureCount(String measureCount) { - this.measureCount = measureCount; - } - - /** - * @return the dimensionCount - */ - public int getDimensionCount() { - return Integer.parseInt(dimensionCount); - } - - /** - * @param dimensionCount the dimensionCount to set - */ - public void setDimensionCount(String dimensionCount) { - this.dimensionCount = dimensionCount; - } - - /** - * @return the complexDimsCount - */ - public int getComplexDimsCount() { - return Integer.parseInt(complexDimsCount); - } - - /** - * @param complexDimsCount the complexDimsCount to set - */ - public void setComplexDimsCount(String complexDimsCount) { - this.complexDimsCount = complexDimsCount; - } - - /** - * @param complexTypeString the complexTypeString to set - */ - public void setComplexTypeString(String complexTypeString) { - this.complexTypeString = complexTypeString; - } - - /** - * @return - */ - public String getNoDictionaryDims() { - return noDictionaryDims; - } - - /** - * @param noDictionaryDims - */ - public void setNoDictionaryDims(String noDictionaryDims) { - this.noDictionaryDims = noDictionaryDims; - } - - /** - * @return the noDictionaryCount - */ - public int getNoDictionaryCount() { - return noDictionaryCount; - } - - /** - * @param noDictionaryCount the noDictionaryCount to set - */ - public void setNoDictionaryCount(int noDictionaryCount) { - this.noDictionaryCount = noDictionaryCount; - } - - public void setColumnGroupsString(String columnGroups) { - this.columnGroupsString = columnGroups; - - } - - public void initialize() { - complexTypes = getComplexTypesMap(complexTypeString); - } - - private Map getComplexTypesMap(String complexTypeString) { - if (null == complexTypeString) { - return new LinkedHashMap<>(); - } - Map complexTypesMap = new LinkedHashMap(); - String[] hierarchies = complexTypeString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER); - for (int i = 0; i < hierarchies.length; i++) { - String[] levels = hierarchies[i].split(CarbonCommonConstants.HASH_SPC_CHARACTER); - String[] levelInfo = levels[0].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - GenericDataType g = levelInfo[1].equals(CarbonCommonConstants.ARRAY) ? - new ArrayDataType(levelInfo[0], "", levelInfo[3]) : - new StructDataType(levelInfo[0], "", levelInfo[3]); - complexTypesMap.put(levelInfo[0], g); - for (int j = 1; j < levels.length; j++) { - levelInfo = levels[j].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - switch (levelInfo[1]) { - case CarbonCommonConstants.ARRAY: - g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3])); - break; - case CarbonCommonConstants.STRUCT: - g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3])); - break; - default: - g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3], - Integer.parseInt(levelInfo[4]))); - } - } - } - return complexTypesMap; - } - - public void setMeasureDataType(String measureDataType) { - this.measureDataType = measureDataType; - } - - /** - * @return - */ - public int getTaskNo() { - return Integer.parseInt(taskNo); - } - - /** - * @param taskNo - */ - public void setTaskNo(String taskNo) { - this.taskNo = taskNo; - } - - /** - * @return - */ - public String getFactTimeStamp() { - return factTimeStamp; - } - - /** - * @param factTimeStamp - */ - public void setFactTimeStamp(String factTimeStamp) { - this.factTimeStamp = factTimeStamp; - } - - /** - * @return partitionId - */ - public String getPartitionID() { - return partitionID; - } - - /** - * @param partitionID - */ - public void setPartitionID(String partitionID) { - this.partitionID = partitionID; - } - - /** - * return segmentId - * - * @return - */ - public String getSegmentId() { - return segmentId; - } - - /** - * set segment Id - * - * @param segmentId - */ - public void setSegmentId(String segmentId) { - this.segmentId = segmentId; - } - - /** - * @return the noDictionaryDimsMapping - */ - public String getNoDictionaryDimsMapping() { - return noDictionaryDimsMapping; - } - - /** - * @param noDictionaryDimsMapping the noDictionaryDimsMapping to set - */ - public void setNoDictionaryDimsMapping(String noDictionaryDimsMapping) { - this.noDictionaryDimsMapping = noDictionaryDimsMapping; - } - /** - * @return isUseInvertedIndex - */ - public String getIsUseInvertedIndex() { - return isUseInvertedIndex; - } - - /** - * @param isUseInvertedIndex the bool array whether use inverted index to set - */ - public void setIsUseInvertedIndex(String isUseInvertedIndex) { - this.isUseInvertedIndex = isUseInvertedIndex; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties deleted file mode 100644 index 3608e57356b..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties +++ /dev/null @@ -1,22 +0,0 @@ -# -# 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. -# -CarbonMDkeyDialog.Shell.Title=Carbon MDkey Generator Step -CarbonMDkeyDialog.TableName.Label=Table Name -CarbonMDkeyDialog.SchemaFieldName.Label=Aggregate Levels -CarbonMDkeyDialog.StoreLocation.Label=Store Location -CarbonMDkeyDialog.NumberOfCores.Label=Number Of Cores \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java deleted file mode 100644 index 45627938ed7..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.merger.step; - -import java.io.File; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.common.logging.impl.StandardLogService; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; - -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStep; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; - -public class CarbonSliceMergerStep extends BaseStep { - - /** - * LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(CarbonSliceMergerStep.class.getName()); - /** - * carbon data writer step data class - */ - private CarbonSliceMergerStepData data; - - /** - * carbon data writer step meta - */ - private CarbonSliceMergerStepMeta meta; - - /** - * readCounter - */ - private long readCounter; - - /** - * writeCounter - */ - private long writeCounter; - - /** - * CarbonSliceMergerStep Constructor - * - * @param stepMeta stepMeta - * @param stepDataInterface stepDataInterface - * @param copyNr copyNr - * @param transMeta transMeta - * @param trans trans - */ - public CarbonSliceMergerStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - super(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - /** - * Perform the equivalent of processing one row. Typically this means - * reading a row from input (getRow()) and passing a row to output - * (putRow)). - * - * @param smi The steps metadata to work with - * @param sdi The steps temporary working data to work with (database - * connections, result sets, caches, temporary variables, etc.) - * @return false if no more rows can be processed or an error occurred. - * @throws KettleException - */ - public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException { - try { - // carbon data writer step meta - meta = (CarbonSliceMergerStepMeta) smi; - StandardLogService.setThreadName(StandardLogService.getPartitionID(meta.getTableName()), - null); - // carbon data writer step data - data = (CarbonSliceMergerStepData) sdi; - - // get row from previous step, blocks when needed! - Object[] row = getRow(); - // if row is null then there is no more incoming data - if (null == row) { - renameFolders(); - - LOGGER.info("Record Procerssed For table: " + meta.getTabelName()); - String logMessage = - "Summary: Carbon Slice Merger Step: Read: " + readCounter + ": Write: " + writeCounter; - LOGGER.info(logMessage); - // step processing is finished - setOutputDone(); - // return false - return false; - } - - if (first) { - first = false; - if (getInputRowMeta() != null) { - this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone()); - this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this); - } - } - readCounter++; - } catch (Exception ex) { - LOGGER.error(ex); - throw new RuntimeException(ex); - } - return true; - } - - private void renameFolders() { - CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal( - meta.getDatabaseName() + File.separator + meta.getTableName() + File.separator + meta - .getTaskNo()); - } - - /** - * Initialize and do work where other steps need to wait for... - * - * @param smi The metadata to work with - * @param sdi The data to initialize - * @return step initialize or not - */ - public boolean init(StepMetaInterface smi, StepDataInterface sdi) { - meta = (CarbonSliceMergerStepMeta) smi; - data = (CarbonSliceMergerStepData) sdi; - return super.init(smi, sdi); - } - - /** - * Dispose of this step: close files, empty logs, etc. - * - * @param smi The metadata to work with - * @param sdi The data to dispose of - */ - public void dispose(StepMetaInterface smi, StepDataInterface sdi) { - meta = (CarbonSliceMergerStepMeta) smi; - data = (CarbonSliceMergerStepData) sdi; - super.dispose(smi, sdi); - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java deleted file mode 100644 index b9890462612..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.merger.step; - -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.step.BaseStepData; -import org.pentaho.di.trans.step.StepDataInterface; - -public class CarbonSliceMergerStepData extends BaseStepData implements StepDataInterface { - /** - * outputRowMeta - */ - private RowMetaInterface outputRowMeta; - - public CarbonSliceMergerStepData() { - super(); - } - - public RowMetaInterface getOutputRowMeta() { - return outputRowMeta; - } - - public void setOutputRowMeta(RowMetaInterface outputRowMeta) { - this.outputRowMeta = outputRowMeta; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java deleted file mode 100644 index 6030a3011a7..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java +++ /dev/null @@ -1,470 +0,0 @@ -/* - * 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.merger.step; - -import java.util.List; -import java.util.Map; - -import org.pentaho.di.core.CheckResult; -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Counter; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.exception.KettleXMLException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.xml.XMLHandler; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.repository.ObjectId; -import org.pentaho.di.repository.Repository; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStepMeta; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.w3c.dom.Node; - -public class CarbonSliceMergerStepMeta extends BaseStepMeta - implements StepMetaInterface, Cloneable { - - /** - * for i18n purposes - */ - private static final Class PKG = CarbonSliceMergerStepMeta.class; - - /** - * table name - */ - private String tabelName; - - /** - * mdkey size - */ - private String mdkeySize; - - /** - * measureCount - */ - private String measureCount; - - /** - * heirAndKeySize - */ - private String heirAndKeySize; - - /** - * databaseName - */ - private String databaseName; - - /** - * tableName - */ - private String tableName; - - /** - * isGroupByEnabled - */ - private String groupByEnabled; - - /** - * aggregatorString - */ - private String aggregatorString; - - /** - * aggregatorClassString - */ - private String aggregatorClassString; - - /** - * factDimLensString - */ - private String factDimLensString; - - private String levelAnddataTypeString; - /** - * partitionID - */ - private String partitionID; - /** - * Id of the load folder - */ - private String segmentId; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - - /** - * CarbonDataWriterStepMeta constructor to initialize this class - */ - public CarbonSliceMergerStepMeta() { - super(); - } - - /** - * set the default value for all the properties - */ - @Override public void setDefault() { - tabelName = ""; - mdkeySize = ""; - measureCount = ""; - heirAndKeySize = ""; - tableName = ""; - databaseName = ""; - groupByEnabled = ""; - aggregatorClassString = ""; - aggregatorString = ""; - factDimLensString = ""; - levelAnddataTypeString = ""; - partitionID = ""; - segmentId = ""; - taskNo = ""; - } - - /** - * Get the XML that represents the values in this step - * - * @return the XML that represents the metadata in this step - * @throws KettleException in case there is a conversion or XML encoding error - */ - public String getXML() { - StringBuilder retval = new StringBuilder(150); - retval.append(" ").append(XMLHandler.addTagValue("TableName", tabelName)); - retval.append(" ").append(XMLHandler.addTagValue("MDKeySize", mdkeySize)); - retval.append(" ").append(XMLHandler.addTagValue("Measurecount", measureCount)); - retval.append(" ").append(XMLHandler.addTagValue("HeirAndKeySize", heirAndKeySize)); - retval.append(" ").append(XMLHandler.addTagValue("tableName", tableName)); - retval.append(" ").append(XMLHandler.addTagValue("databaseName", databaseName)); - retval.append(" ").append(XMLHandler.addTagValue("isGroupByEnabled", groupByEnabled)); - retval.append(" ") - .append(XMLHandler.addTagValue("aggregatorClassString", aggregatorClassString)); - retval.append(" ").append(XMLHandler.addTagValue("aggregatorString", aggregatorString)); - retval.append(" ").append(XMLHandler.addTagValue("factDimLensString", factDimLensString)); - retval.append(" ") - .append(XMLHandler.addTagValue("levelAnddataTypeString", levelAnddataTypeString)); - retval.append(" ").append(XMLHandler.addTagValue("partitionID", partitionID)); - retval.append(" ").append(XMLHandler.addTagValue("segmentId", segmentId)); - retval.append(" ").append(XMLHandler.addTagValue("taskNo", taskNo)); - return retval.toString(); - } - - /** - * Load the values for this step from an XML Node - * - * @param stepnode the Node to get the info from - * @param databases The available list of databases to reference to - * @param counters Counters to reference. - * @throws KettleXMLException When an unexpected XML error occurred. (malformed etc.) - */ - @Override public void loadXML(Node stepnode, List databases, - Map counters) throws KettleXMLException { - try { - databaseName = XMLHandler.getTagValue(stepnode, "databaseName"); - tabelName = XMLHandler.getTagValue(stepnode, "TableName"); - mdkeySize = XMLHandler.getTagValue(stepnode, "MDKeySize"); - measureCount = XMLHandler.getTagValue(stepnode, "Measurecount"); - heirAndKeySize = XMLHandler.getTagValue(stepnode, "HeirAndKeySize"); - tableName = XMLHandler.getTagValue(stepnode, "tableName"); - groupByEnabled = XMLHandler.getTagValue(stepnode, "isGroupByEnabled"); - aggregatorClassString = XMLHandler.getTagValue(stepnode, "aggregatorClassString"); - aggregatorString = XMLHandler.getTagValue(stepnode, "aggregatorString"); - factDimLensString = XMLHandler.getTagValue(stepnode, "factDimLensString"); - levelAnddataTypeString = XMLHandler.getTagValue(stepnode, "levelAnddataTypeString"); - partitionID = XMLHandler.getTagValue(stepnode, "partitionID"); - segmentId = XMLHandler.getTagValue(stepnode, "segmentId"); - taskNo = XMLHandler.getTagValue(stepnode, "taskNo"); - } catch (Exception e) { - throw new KettleXMLException("Unable to read step info from XML node", e); - } - } - - /** - * Save the steps data into a Kettle repository - * - * @param rep The Kettle repository to save to - * @param idTransformation The transformation ID - * @param idStep The step ID - * @throws KettleException When an unexpected error occurred (database, network, etc) - */ - @Override public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep) - throws KettleException { - try { - rep.saveStepAttribute(idTransformation, idStep, "TableName", tabelName); //$NON-NLS-1$ - rep.saveStepAttribute(idTransformation, idStep, "MDKeySize", mdkeySize); //$NON-NLS-1$ - rep.saveStepAttribute(idTransformation, idStep, "Measurecount", measureCount); - rep.saveStepAttribute(idTransformation, idStep, "HeirAndKeySize", - heirAndKeySize); //$NON-NLS-1$ - rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName); //$NON-NLS-1$ - rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName); //$NON-NLS-1$ - rep.saveStepAttribute(idTransformation, idStep, "isGroupByEnabled", groupByEnabled); - rep.saveStepAttribute(idTransformation, idStep, "aggregatorClassString", - aggregatorClassString); - rep.saveStepAttribute(idTransformation, idStep, "aggregatorString", aggregatorString); - rep.saveStepAttribute(idTransformation, idStep, "factDimLensString", factDimLensString); - rep.saveStepAttribute(idTransformation, idStep, "levelAnddataTypeString", - levelAnddataTypeString); - rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID); - rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId); - rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo); - } catch (Exception e) { - throw new KettleException( - BaseMessages.getString(PKG, "TemplateStep.Exception.UnableToSaveStepInfoToRepository") - + idStep, e); - } - } - - /** - * Make an exact copy of this step, make sure to explicitly copy Collections - * etc. - * - * @return an exact copy of this step - */ - public Object clone() { - Object retval = super.clone(); - return retval; - } - - /** - * Read the steps information from a Kettle repository - * - * @param rep The repository to read from - * @param idStep The step ID - * @param databases The databases to reference - * @param counters The counters to reference - * @throws KettleException When an unexpected error occurred (database, network, etc) - */ - @Override public void readRep(Repository rep, ObjectId idStep, List databases, - Map counters) throws KettleException { - try { - tabelName = rep.getStepAttributeString(idStep, "TableName"); - mdkeySize = rep.getStepAttributeString(idStep, "MDKeySize"); - measureCount = rep.getStepAttributeString(idStep, "Measurecount"); - heirAndKeySize = rep.getStepAttributeString(idStep, "HeirAndKeySize"); - databaseName = rep.getStepAttributeString(idStep, "databaseName"); - tableName = rep.getStepAttributeString(idStep, "tableName"); - groupByEnabled = rep.getStepAttributeString(idStep, "isGroupByEnabled"); - aggregatorClassString = rep.getStepAttributeString(idStep, "aggregatorClassString"); - aggregatorString = rep.getStepAttributeString(idStep, "aggregatorString"); - factDimLensString = rep.getStepAttributeString(idStep, "factDimLensString"); - levelAnddataTypeString = rep.getStepAttributeString(idStep, "levelAnddataTypeString"); - partitionID = rep.getStepAttributeString(idStep, "partitionID"); - segmentId = rep.getStepAttributeString(idStep, "segmentId"); - taskNo = rep.getStepAttributeString(idStep, "taskNo"); - } catch (Exception exception) { - throw new KettleException(BaseMessages - .getString(PKG, "CarbonDataWriterStepMeta.Exception.UnexpectedErrorInReadingStepInfo"), - exception); - } - - } - - /** - * Get the executing step, needed by Trans to launch a step. - * - * @param stepMeta The step info - * @param stepDataInterface the step data interface linked to this step. Here the step can - * store temporary data, database connections, etc. - * @param copyNr The copy nr to get - * @param transMeta The transformation info - * @param trans The launching transformation - */ - @Override public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, - int copyNr, TransMeta transMeta, Trans trans) { - return new CarbonSliceMergerStep(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - /** - * Checks the settings of this step and puts the findings in a remarks List. - * - * @param remarks The list to put the remarks in @see - * org.pentaho.di.core.CheckResult - * @param stepMeta The stepMeta to help checking - * @param prev The fields coming from the previous step - * @param input The input step names - * @param output The output step names - * @param info The fields that are used as information by the step - */ - @Override public void check(List remarks, TransMeta transMeta, - StepMeta stepMeta, RowMetaInterface prev, String[] input, String[] output, - RowMetaInterface info) { - - CheckResult checkResVal; - - // See if we have input streams leading to this step! - if (input.length > 0) { - checkResVal = - new CheckResult(CheckResult.TYPE_RESULT_OK, "Step is receiving info from other steps.", - stepMeta); - remarks.add(checkResVal); - } else { - checkResVal = - new CheckResult(CheckResult.TYPE_RESULT_ERROR, "No input received from other steps!", - stepMeta); - remarks.add(checkResVal); - } - - } - - /** - * Get a new instance of the appropriate data class. This data class - * implements the StepDataInterface. It basically contains the persisting - * data that needs to live on, even if a worker thread is terminated. - * - * @return The appropriate StepDataInterface class. - */ - @Override public StepDataInterface getStepData() { - return new CarbonSliceMergerStepData(); - } - - /** - * This method will return the table name - * - * @return tabelName - */ - public String getTabelName() { - return tabelName; - } - - /** - * This method will set the table name - * - * @param tabelName - */ - public void setTabelName(String tabelName) { - this.tabelName = tabelName; - } - - /** - * This method will be used to set the mdkey - * - * @param mdkeySize - */ - public void setMdkeySize(String mdkeySize) { - this.mdkeySize = mdkeySize; - } - - /** - * This method will be used to set the measure count - * - * @param measureCount - */ - public void setMeasureCount(String measureCount) { - this.measureCount = measureCount; - } - - /** - * This method will be used to set the heir and key size string - * - * @param heirAndKeySize - */ - public void setHeirAndKeySize(String heirAndKeySize) { - this.heirAndKeySize = heirAndKeySize; - } - - /** - * @return the databaseName - */ - public String getDatabaseName() { - return databaseName; - } - - /** - * @param databaseName the databaseName to set - */ - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - /** - * @return the tableName - */ - public String getTableName() { - return tableName; - } - - /** - * @param tableName the tableName to set - */ - public void setTableName(String tableName) { - this.tableName = tableName; - } - - /** - * @param isGroupByEnabled the isGroupByEnabled to set - */ - public void setGroupByEnabled(String isGroupByEnabled) { - this.groupByEnabled = isGroupByEnabled; - } - - /** - * @param aggregatorString the aggregatorString to set - */ - public void setAggregatorString(String aggregatorString) { - this.aggregatorString = aggregatorString; - } - - /** - * @param aggregatorClassString the aggregatorClassString to set - */ - public void setAggregatorClassString(String aggregatorClassString) { - this.aggregatorClassString = aggregatorClassString; - } - - /** - * @param factDimLensString1 the factDimLensString to set - */ - public void setFactDimLensString(String factDimLensString1) { - this.factDimLensString = factDimLensString1; - } - - public void setLevelAnddataTypeString(String levelAnddataTypeString) { - this.levelAnddataTypeString = levelAnddataTypeString; - } - - /** - * @param partitionID - */ - public void setPartitionID(String partitionID) { - this.partitionID = partitionID; - } - - /** - * set segment Id - * @param segmentId - */ - public void setSegmentId(String segmentId) { - this.segmentId = segmentId; - } - - /** - * @param taskNo - */ - public void setTaskNo(String taskNo) { - this.taskNo = taskNo; - } - - /** - * @return - */ - public String getTaskNo() { - return taskNo; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java index 433f8dbd874..ad965782fec 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java @@ -85,7 +85,7 @@ public void initialize(SortParameters sortParameters) { sortParameters.getDimColCount(), sortParameters.getComplexDimColCount(), sortParameters.getMeasureColCount(), sortParameters.getNoDictionaryCount(), sortParameters.getAggType(), - sortParameters.getNoDictionaryDimnesionColumn(), sortParameters.isUseKettle()); + sortParameters.getNoDictionaryDimnesionColumn()); } @Override diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java index 813d83dcda6..e3049d21b69 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java @@ -139,8 +139,7 @@ private SingleThreadFinalSortFilesMerger getFinalMerger(String bucketId) { new SingleThreadFinalSortFilesMerger(dataFolderLocation, sortParameters.getTableName(), sortParameters.getDimColCount(), sortParameters.getComplexDimColCount(), sortParameters.getMeasureColCount(), sortParameters.getNoDictionaryCount(), - sortParameters.getAggType(), sortParameters.getNoDictionaryDimnesionColumn(), - sortParameters.isUseKettle()); + sortParameters.getAggType(), sortParameters.getNoDictionaryDimnesionColumn()); return finalMerger; } diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java index 7eacd085759..f10e73aed32 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java +++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java @@ -134,7 +134,7 @@ private String getStoreLocation(CarbonTableIdentifier tableIdentifier, String pa "Error while initializing data handler : " + e.getMessage()); } catch (Exception e) { LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl"); - throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage()); + throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e); } return null; } diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java deleted file mode 100644 index 27c37181ca1..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.schema.metadata; - -import java.io.Serializable; -import java.util.Arrays; - -public class ArrayWrapper implements Serializable { - - /** - * Comment for serialVersionUID - */ - private static final long serialVersionUID = -2016551342632572869L; - - /** - * data - */ - private int[] data; - - public ArrayWrapper(int[] data) { - if (data == null) { - throw new IllegalArgumentException(); - } - this.data = data; - } - - @Override public boolean equals(Object other) { - if (other instanceof ArrayWrapper) { - return Arrays.equals(data, ((ArrayWrapper) other).data); - } else { - return false; - } - - } - - @Override public int hashCode() { - return Arrays.hashCode(data); - } - - public int[] getData() { - return data; - } - - public void setData(int[] data) { - this.data = data; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java deleted file mode 100644 index 2484002dd90..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.schema.metadata; - -import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.util.DataTypeUtil; - -/** - * Class holds the common column schema details needed for the data load - */ -public class ColumnSchemaDetails { - - /** - * column Name - */ - private String columnName; - /** - * column datatype - */ - private DataType columnType; - /** - * boolean to identify direct dictionary column - */ - private Boolean isDirectDictionary; - - /** - * Constructor to initialize object from the input string separated by comma (,) - * - * @param input - */ - ColumnSchemaDetails(String input) { - String[] splits = input.split(","); - columnName = splits[0]; - columnType = DataTypeUtil.getDataType(splits[1]); - isDirectDictionary = Boolean.parseBoolean(splits[2]); - } - - /** - * Constructor to initialize the ColumnSchemaDetails - * - * @param columnName - * @param columnType - * @param isDirectDictionary - */ - public ColumnSchemaDetails(String columnName, DataType columnType, Boolean isDirectDictionary) { - this.columnName = columnName; - this.columnType = columnType; - this.isDirectDictionary = isDirectDictionary; - - } - - /** - * returns the ColumnName - * - * @return - */ - public String getColumnName() { - return columnName; - } - - /** - * returns the dataType of the column - * - * @return - */ - public DataType getColumnType() { - return columnType; - } - - /** - * returns boolean value to identify direct dictionary - * - * @return - */ - public Boolean isDirectDictionary() { - return isDirectDictionary; - } - - /** - * @return - */ - public String toString() { - return columnName + "," + columnType + "," + isDirectDictionary; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java deleted file mode 100644 index 554dd06a892..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.schema.metadata; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; - -/** - * Wrapper class to hold the columnschema details - */ -public class ColumnSchemaDetailsWrapper { - - /** - * Map of the ColumnSchemaDetails - */ - private Map columnSchemaDetailsMap; - - /** - * return the string object - * - * @return - */ - public String toString() { - StringBuilder builder = new StringBuilder(); - Set> entries = columnSchemaDetailsMap.entrySet(); - Iterator> iterator = entries.iterator(); - - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - builder.append(entry.getKey()); - builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - builder.append(entry.getValue().toString()); - if (iterator.hasNext()) { - builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - } - return builder.toString(); - } - - /** - * default constructor - */ - public ColumnSchemaDetailsWrapper() { - - } - - /** - * Constructor take serialized string as input and populates the List of columnschema details - * - * @param input - */ - public ColumnSchemaDetailsWrapper(String input) { - columnSchemaDetailsMap = new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - String[] split = input.split(CarbonCommonConstants.HASH_SPC_CHARACTER); - for (int i = 0; i < split.length; i++) { - String key = split[i++]; - ColumnSchemaDetails details = new ColumnSchemaDetails(split[i]); - columnSchemaDetailsMap.put(key, details); - } - } - - /** - * returns ColumnSchemaDetails of all columns - * - * @return - */ - public Map getColumnSchemaDetailsMap() { - return columnSchemaDetailsMap; - } - - /** - * sets the map of column schema - * - * @param columnSchemaDetailsMap - */ - public void setColumnSchemaDetailsMap(Map columnSchemaDetailsMap) { - this.columnSchemaDetailsMap = columnSchemaDetailsMap; - } - - /** - * returns the columnSchemaDetails of requested column - * - * @param key - * @return - */ - public ColumnSchemaDetails get(String key) { - return columnSchemaDetailsMap.get(key); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java deleted file mode 100644 index b8b8bcd29f6..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java +++ /dev/null @@ -1,430 +0,0 @@ -/* - * 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.schema.metadata; - -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.processing.datatypes.GenericDataType; - -public class ColumnsInfo { - - /** - * Indices for dimensions in the record. Doesn't include any properties. - */ - private int[] dims; - - /** - * Map - */ - private Map keyGenerators; - - /** - * Hierarchy table names (Same will be file names for file store or - * incremental load) - */ - private Set hierTables; - - /** - * Batch size configured in transformation - */ - private int batchSize; - - /** - * To decide it is data load for aggregate table or not. - */ - private boolean isAggregateLoad; - - /** - * Store type DB or file based ? - */ - private String storeType; - - /** - * column Names for dimensions. Which will be used as table name for store - */ - private String[] dimColNames; - - /** - * baseStoreLocation - */ - private String baseStoreLocation; - - /** - * Maximum possible surrogate key for dimension possible based on - * cardinality value in schema definition - */ - private int[] maxKeys; - - /** - * Dimension Index, Properties indices in the tuple. - * [0] - [2,3] - 2 Props at indices 2 & 3 [1] - [4,7,8] - 3 props at indices - * 4,7, & 8 [2] - [] - No props - */ - private int[][] propIndx; - - /** - * Dimension Index, Property column names from table. - * [0] - [col2,col3] [1] - [col4,col7,col8] [2] - [] - */ - private List[] propColumns; - - /** - * timDimIndex - */ - private int timDimIndex; - - /** - * timDimIndexEnd - */ - private int timDimIndexEnd; - - /** - * timeOrdinalIndices - */ - private int[] timeOrdinalIndices; - - /** - * timeOrdinalCols - */ - private String[] timeOrdinalCols; - - /** - * propTypes - */ - private List[] propTypes; - - /** - * dimHierRel - */ - private String[] dimHierRel; - - /** - * tableName - */ - private String tableName; - - /** - * Primary key Map - */ - private Map primaryKeyMap; - - /** - * measureColumns - */ - private String[] measureColumns; - - private boolean[] dimsPresent; - - private String databaseName; - - private Map complexTypesMap; - - /** - * column Ids of dimensions in a table - */ - private String[] dimensionColumnIds; - - /** - * wrapper object having the columnSchemaDetails - */ - private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper; - - private Map> columnProperties; - - public Map getComplexTypesMap() { - return complexTypesMap; - } - - public void setComplexTypesMap(Map complexTypesMap) { - this.complexTypesMap = complexTypesMap; - } - - /** - * @return Returns the dims. - */ - public int[] getDims() { - return dims; - } - - /** - * @param dims The dims to set. - */ - public void setDims(int[] dims) { - this.dims = dims; - } - - /** - * @return Returns the keyGenerators. - */ - public Map getKeyGenerators() { - return keyGenerators; - } - - /** - * @param keyGenerators The keyGenerators to set. - */ - public void setKeyGenerators(Map keyGenerators) { - this.keyGenerators = keyGenerators; - } - - /** - * @return Returns the tableName. - */ - public String getTableName() { - return tableName; - } - - /** - * @param tableName The tableName to set. - */ - public void setTableName(String tableName) { - this.tableName = tableName; - } - - /** - * @return Returns the hierTables. - */ - public Set getHierTables() { - return hierTables; - } - - /** - * @param hierTables The hierTables to set. - */ - public void setHierTables(Set hierTables) { - this.hierTables = hierTables; - } - - /** - * @param batchSize The batchSize to set. - */ - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - public void setAggregateLoad(boolean isAggregateLoad) { - this.isAggregateLoad = isAggregateLoad; - } - - /** - * @param storeType The storeType to set. - */ - public void setStoreType(String storeType) { - this.storeType = storeType; - } - - /** - * @return Returns the dimColNames. - */ - public String[] getDimColNames() { - return dimColNames; - } - - /** - * @param dimColNames The dimColNames to set. - */ - public void setDimColNames(String[] dimColNames) { - this.dimColNames = dimColNames; - } - - /** - * @return Returns the maxKeys. - */ - public int[] getMaxKeys() { - return maxKeys; - } - - /** - * @param maxKeys The maxKeys to set. - */ - public void setMaxKeys(int[] maxKeys) { - this.maxKeys = maxKeys; - } - - /** - * @return Returns the propIndx. - */ - public int[][] getPropIndx() { - return propIndx; - } - - /** - * @param propIndx The propIndx to set. - */ - public void setPropIndx(int[][] propIndx) { - this.propIndx = propIndx; - } - - /** - * @param propColumns The propColumns to set. - */ - public void setPropColumns(List[] propColumns) { - this.propColumns = propColumns; - } - - /** - * @return Returns the timDimIndex. - */ - public int getTimDimIndex() { - return timDimIndex; - } - - /** - * @param timDimIndex The timDimIndex to set. - */ - public void setTimDimIndex(int timDimIndex) { - this.timDimIndex = timDimIndex; - } - - /** - * @return Returns the timDimIndexEnd. - */ - public int getTimDimIndexEnd() { - return timDimIndexEnd; - } - - /** - * @return Returns the timeOrdinalIndices. - */ - public int[] getTimeOrdinalIndices() { - return timeOrdinalIndices; - } - - /** - * @param timeOrdinalIndices The timeOrdinalIndices to set. - */ - public void setTimeOrdinalIndices(int[] timeOrdinalIndices) { - this.timeOrdinalIndices = timeOrdinalIndices; - } - - /** - * @param timeOrdinalCols The timeOrdinalCols to set. - */ - public void setTimeOrdinalCols(String[] timeOrdinalCols) { - this.timeOrdinalCols = timeOrdinalCols; - } - - /** - * @param propTypes The propTypes to set. - */ - public void setPropTypes(List[] propTypes) { - this.propTypes = propTypes; - } - - /** - * @return Returns the baseStoreLocation. - */ - public String getBaseStoreLocation() { - return baseStoreLocation; - } - - /** - * @param baseStoreLocation The baseStoreLocation to set. - */ - public void setBaseStoreLocation(String baseStoreLocation) { - this.baseStoreLocation = baseStoreLocation; - } - - /** - * @param dimHierRel The dimHierRel to set. - */ - public void setDimHierRel(String[] dimHierRel) { - this.dimHierRel = dimHierRel; - } - - /** - * @return Returns the primaryKeyMap. - */ - public Map getPrimaryKeyMap() { - return primaryKeyMap; - } - - /** - * @param primaryKeyMap The primaryKeyMap to set. - */ - public void setPrimaryKeyMap(Map primaryKeyMap) { - this.primaryKeyMap = primaryKeyMap; - } - - /** - * getDimsPresent - * - * @return boolean[] - */ - public boolean[] getDimsPresent() { - return dimsPresent; - } - - /** - * @param measureColumns The measureColumns to set. - */ - public void setMeasureColumns(String[] measureColumns) { - this.measureColumns = measureColumns; - } - - public String getDatabaseName() { - return databaseName; - } - - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - /** - * @return column Ids - */ - public String[] getDimensionColumnIds() { - return dimensionColumnIds; - } - - /** - * @param dimensionColumnIds column Ids for dimensions in a table - */ - public void setDimensionColumnIds(String[] dimensionColumnIds) { - this.dimensionColumnIds = dimensionColumnIds; - } - - /** - * returns wrapper object having the columnSchemaDetails - * - * @return - */ - public ColumnSchemaDetailsWrapper getColumnSchemaDetailsWrapper() { - return columnSchemaDetailsWrapper; - } - - /** - * set the wrapper object having the columnSchemaDetails - * - * @param columnSchemaDetailsWrapper - */ - public void setColumnSchemaDetailsWrapper(ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper) { - this.columnSchemaDetailsWrapper = columnSchemaDetailsWrapper; - } - - public void setColumnProperties(Map> columnProperties) { - this.columnProperties = columnProperties; - } - - public Map getColumnProperties(String columnName) { - return this.columnProperties.get(columnName); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java deleted file mode 100644 index 32f096d6206..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.schema.metadata; - -import java.util.Map; - -public class HierarchiesInfo { - - /** - * hierarichieName - */ - private String hierarichieName; - - /** - * columnPropMap - */ - private Map columnPropMap; - - public String getHierarichieName() { - return hierarichieName; - } - - public void setHierarichieName(String hierarichieName) { - this.hierarichieName = hierarichieName; - } - - public Map getColumnPropMap() { - return columnPropMap; - } - - public void setColumnPropMap(Map columnPropMap) { - this.columnPropMap = columnPropMap; - } - - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java deleted file mode 100644 index c01d800a4ae..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java +++ /dev/null @@ -1,80 +0,0 @@ -/* - * 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.schema.metadata; - -/** - * This class is to hold the key value pair of properties needed while dataload. - */ -public class TableOption { - /** - * option key name - */ - private String optionKey; - /** - * option key value - */ - private String optionValue; - - /** - * the constructor to initialize the key value pair TableOption instance - * - * @param optionKey - * @param optionValue - */ - public TableOption(String optionKey, String optionValue) { - this.optionKey = optionKey; - this.optionValue = optionValue; - } - - /** - * constructor to init from te string separated by comma(,) - * - * @param str - */ - public TableOption(String str) { - //passing 2 to split the key value pair having empty value for the corresponding key. - String[] split = str.split(",", 2); - this.optionKey = split[0]; - this.optionValue = split[1]; - } - - /** - * returns options key - * - * @return - */ - public String getOptionKey() { - return optionKey; - } - - /** - * returns options value - * - * @return - */ - public String getOptionValue() { - return optionValue; - } - - /** - * @return - */ - public String toString() { - return optionKey + "," + optionValue; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java deleted file mode 100644 index 02f797ac47a..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.schema.metadata; - -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.Set; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; - -/** - * The class hold the table option details being used while dataload - */ -public class TableOptionWrapper { - /** - * map holds the table options - */ - private static final Map mapOFOptions = - new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - private static TableOptionWrapper tableOptionWrapper = new TableOptionWrapper(); - - /** - * to initialize the wrapper object - */ - private TableOptionWrapper() { - } - - /** - * @param input - */ - public static void populateTableOptions(String input) { - String[] split = - null != input ? input.split(CarbonCommonConstants.HASH_SPC_CHARACTER) : new String[0]; - for (String str : split) { - TableOption tableOption = new TableOption(str); - mapOFOptions.put(tableOption.getOptionKey(), tableOption); - } - } - - /** - * @param input - */ - public static void setTableOption(String input) { - if (null != input) { - TableOption tableOption = new TableOption(input); - mapOFOptions.put(tableOption.getOptionKey(), tableOption); - } - } - - /** - * returns TableOptionWrapper instance - * - * @return - */ - public static TableOptionWrapper getTableOptionWrapperInstance() { - return tableOptionWrapper; - } - - /** - * returns the options key value - * return null if the key is not found in the map - * - * @param key - * @return - */ - public String get(String key) { - TableOption tableOption = mapOFOptions.get(key); - return null != tableOption ? tableOption.getOptionValue() : null; - } - - /** - * return the string object - * - * @return - */ - public String toString() { - StringBuilder builder = new StringBuilder(); - Set> entries = mapOFOptions.entrySet(); - Iterator> iterator = entries.iterator(); - - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - builder.append(entry.getValue().toString()); - builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - return builder.toString(); - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java index 1b166757acb..0ac2d5c6a4a 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java @@ -87,8 +87,6 @@ public class IntermediateFileMerger implements Callable { private File outPutFile; - private boolean useKettle; - private boolean[] noDictionarycolumnMapping; /** @@ -100,7 +98,6 @@ public IntermediateFileMerger(SortParameters mergerParameters, File[] intermedia this.fileCounter = intermediateFiles.length; this.intermediateFiles = intermediateFiles; this.outPutFile = outPutFile; - this.useKettle = mergerParameters.isUseKettle(); noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn(); } @@ -111,14 +108,8 @@ public IntermediateFileMerger(SortParameters mergerParameters, File[] intermedia try { startSorting(); initialize(); - if (useKettle) { - while (hasNext()) { - writeDataTofile(next()); - } - } else { - while (hasNext()) { - writeDataTofileWithOutKettle(next()); - } + while (hasNext()) { + writeDataTofile(next()); } if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) { if (entryCount > 0) { @@ -260,8 +251,7 @@ private void startSorting() throws CarbonSortKeyAndGroupByException { new SortTempFileChunkHolder(tempFile, mergerParameters.getDimColCount(), mergerParameters.getComplexDimColCount(), mergerParameters.getMeasureColCount(), mergerParameters.getFileBufferSize(), mergerParameters.getNoDictionaryCount(), - mergerParameters.getAggType(), mergerParameters.getNoDictionaryDimnesionColumn(), - mergerParameters.isUseKettle()); + mergerParameters.getAggType(), mergerParameters.getNoDictionaryDimnesionColumn()); // initialize sortTempFileChunkHolder.initialize(); @@ -283,7 +273,7 @@ private void startSorting() throws CarbonSortKeyAndGroupByException { */ private void createRecordHolderQueue(File[] listFiles) { // creating record holder heap - this.recordHolderHeap = new PriorityQueue(listFiles.length); + this.recordHolderHeap = new PriorityQueue<>(listFiles.length); } /** @@ -309,76 +299,9 @@ private boolean hasNext() { /** * Below method will be used to write data to file * - * TODO Remove it after kettle is removed - * * @throws CarbonSortKeyAndGroupByException problem while writing */ private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException { - if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) { - if (entryCount == 0) { - records = new Object[totalSize][]; - records[entryCount++] = row; - return; - } - - records[entryCount++] = row; - if (entryCount == totalSize) { - this.writer.writeSortTempFile(records); - entryCount = 0; - records = new Object[totalSize][]; - } - return; - } - try { - int fieldIndex = 0; - char[] aggType = mergerParameters.getAggType(); - - for (int counter = 0; counter < mergerParameters.getDimColCount(); counter++) { - stream.writeInt((Integer) NonDictionaryUtil.getDimension(fieldIndex++, row)); - } - - // added for high card also - if ((mergerParameters.getNoDictionaryCount() + mergerParameters - .getComplexDimColCount()) > 0) { - stream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row)); - } - - fieldIndex = 0; - for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) { - if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) { - stream.write((byte) 1); - if (aggType[counter] == CarbonCommonConstants.BYTE_VALUE_MEASURE) { - Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeDouble(val); - } else if (aggType[counter] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) { - Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeDouble(val); - } else if (aggType[counter] == CarbonCommonConstants.BIG_INT_MEASURE) { - Long val = (Long) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeLong(val); - } else if (aggType[counter] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeInt(bigDecimalInBytes.length); - stream.write(bigDecimalInBytes); - } - } else { - stream.write((byte) 0); - } - - fieldIndex++; - } - - } catch (IOException e) { - throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e); - } - } - - /** - * Below method will be used to write data to file - * - * @throws CarbonSortKeyAndGroupByException problem while writing - */ - private void writeDataTofileWithOutKettle(Object[] row) throws CarbonSortKeyAndGroupByException { if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) { if (entryCount == 0) { records = new Object[totalSize][]; diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java index 794935debc6..9b5a8509e8a 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java @@ -38,7 +38,6 @@ import org.apache.carbondata.core.util.DataTypeUtil; import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException; import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; -import org.apache.carbondata.processing.util.NonDictionaryUtil; public class SortDataRows { /** @@ -193,19 +192,10 @@ public void startSorting() throws CarbonSortKeyAndGroupByException { toSort = new Object[entryCount][]; System.arraycopy(recordHolderList, 0, toSort, 0, entryCount); - if (parameters.isUseKettle()) { - if (parameters.getNoDictionaryCount() > 0) { - Arrays.sort(toSort, new RowComparator(parameters.getNoDictionaryDimnesionColumn(), - parameters.getNoDictionaryCount())); - } else { - Arrays.sort(toSort, new RowComparatorForNormalDims(parameters.getDimColCount())); - } + if (parameters.getNoDictionaryCount() > 0) { + Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionaryDimnesionColumn())); } else { - if (parameters.getNoDictionaryCount() > 0) { - Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionaryDimnesionColumn())); - } else { - Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getDimColCount())); - } + Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getDimColCount())); } recordHolderList = toSort; @@ -233,11 +223,7 @@ private void writeDataTofile(Object[][] recordHolderList, int entryCountLocal, F writeSortTempFile(recordHolderList, entryCountLocal, file); return; } - if (parameters.isUseKettle()) { - writeData(recordHolderList, entryCountLocal, file); - } else { - writeDataWithOutKettle(recordHolderList, entryCountLocal, file); - } + writeData(recordHolderList, entryCountLocal, file); } private void writeSortTempFile(Object[][] recordHolderList, int entryCountLocal, File file) @@ -258,70 +244,9 @@ private void writeSortTempFile(Object[][] recordHolderList, int entryCountLocal, } } - // TODO Remove it after kettle got removed private void writeData(Object[][] recordHolderList, int entryCountLocal, File file) throws CarbonSortKeyAndGroupByException { DataOutputStream stream = null; - try { - // open stream - stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file), - parameters.getFileWriteBufferSize())); - - // write number of entries to the file - stream.writeInt(entryCountLocal); - int dimColCount = parameters.getDimColCount(); - int combinedDimCount = parameters.getNoDictionaryCount() + parameters.getComplexDimColCount(); - char[] aggType = parameters.getAggType(); - Object[] row = null; - for (int i = 0; i < entryCountLocal; i++) { - // get row from record holder list - row = recordHolderList[i]; - int fieldIndex = 0; - - for (int dimCount = 0; dimCount < dimColCount; dimCount++) { - stream.writeInt(NonDictionaryUtil.getDimension(fieldIndex++, row)); - } - - // if any high cardinality dims are present then write it to the file. - - if (combinedDimCount > 0) { - stream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row)); - } - - // as measures are stored in separate array. - fieldIndex = 0; - for (int mesCount = 0; mesCount < parameters.getMeasureColCount(); mesCount++) { - if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) { - stream.write((byte) 1); - if (aggType[mesCount] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) { - Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeDouble(val); - } else if (aggType[mesCount] == CarbonCommonConstants.BIG_INT_MEASURE) { - Long val = (Long) NonDictionaryUtil.getMeasure(fieldIndex, row); - stream.writeLong(val); - } else if (aggType[mesCount] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - BigDecimal val = (BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row); - byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val); - stream.writeInt(bigDecimalInBytes.length); - stream.write(bigDecimalInBytes); - } - } else { - stream.write((byte) 0); - } - fieldIndex++; - } - } - } catch (IOException e) { - throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e); - } finally { - // close streams - CarbonUtil.closeStreams(stream); - } - } - - private void writeDataWithOutKettle(Object[][] recordHolderList, int entryCountLocal, File file) - throws CarbonSortKeyAndGroupByException { - DataOutputStream stream = null; try { // open stream stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file), @@ -460,23 +385,12 @@ public DataSorterAndWriter(Object[][] recordHolderArray) { @Override public Void call() throws Exception { try { long startTime = System.currentTimeMillis(); - if (parameters.isUseKettle()) { - if (parameters.getNoDictionaryCount() > 0) { - Arrays.sort(recordHolderArray, - new RowComparator(parameters.getNoDictionaryDimnesionColumn(), - parameters.getNoDictionaryCount())); - } else { - Arrays.sort(recordHolderArray, - new RowComparatorForNormalDims(parameters.getDimColCount())); - } + if (parameters.getNoDictionaryCount() > 0) { + Arrays.sort(recordHolderArray, + new NewRowComparator(parameters.getNoDictionaryDimnesionColumn())); } else { - if (parameters.getNoDictionaryCount() > 0) { - Arrays.sort(recordHolderArray, - new NewRowComparator(parameters.getNoDictionaryDimnesionColumn())); - } else { - Arrays.sort(recordHolderArray, - new NewRowComparatorForNormalDims(parameters.getDimColCount())); - } + Arrays.sort(recordHolderArray, + new NewRowComparatorForNormalDims(parameters.getDimColCount())); } // create a new file every time diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java index a52ebb23587..d42dc324e5e 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java @@ -114,11 +114,6 @@ public class SortParameters { private int numberOfCores; - /** - * TODO Temporary conf , it will be removed after kettle removal. - */ - private boolean useKettle = true; - public SortParameters getCopy() { SortParameters parameters = new SortParameters(); parameters.tempFileLocation = tempFileLocation; @@ -143,7 +138,6 @@ public SortParameters getCopy() { parameters.taskNo = taskNo; parameters.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn; parameters.numberOfCores = numberOfCores; - parameters.useKettle = useKettle; return parameters; } @@ -323,14 +317,6 @@ public void setNumberOfCores(int numberOfCores) { this.numberOfCores = numberOfCores; } - public boolean isUseKettle() { - return useKettle; - } - - public void setUseKettle(boolean useKettle) { - this.useKettle = useKettle; - } - public static SortParameters createSortParameters(CarbonDataLoadConfiguration configuration) { SortParameters parameters = new SortParameters(); CarbonTableIdentifier tableIdentifier = @@ -432,7 +418,6 @@ public static SortParameters createSortParameters(CarbonDataLoadConfiguration co char[] aggType = CarbonDataProcessorUtil .getAggType(configuration.getMeasureCount(), configuration.getMeasureFields()); parameters.setAggType(aggType); - parameters.setUseKettle(false); return parameters; } diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java index fef8c9d98da..ae014047656 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java +++ b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java @@ -23,7 +23,6 @@ import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -32,7 +31,6 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.constants.IgnoreDictionary; import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.CarbonUtil; @@ -134,9 +132,6 @@ public class SortTempFileChunkHolder implements Comparable 0) { - short lengthOfByteArray = stream.readShort(); - ByteBuffer buff = ByteBuffer.allocate(lengthOfByteArray + 2); - buff.putShort(lengthOfByteArray); - byte[] byteArr = new byte[lengthOfByteArray]; - stream.readFully(byteArr); - - buff.put(byteArr); - finalByteArr = buff.array(); - - } - - index = 0; - // read measure values - for (int i = 0; i < this.measureCount; i++) { - if (stream.readByte() == 1) { - if (aggType[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) { - measures[index++] = stream.readDouble(); - } else if (aggType[i] == CarbonCommonConstants.BIG_INT_MEASURE) { - measures[index++] = stream.readLong(); - } else { - int len = stream.readInt(); - byte[] buff = new byte[len]; - stream.readFully(buff); - measures[index++] = buff; - } - } else { - measures[index++] = null; - } - } - - NonDictionaryUtil.prepareOutObj(holder, dim, finalByteArr, measures); - - // increment number if record read - this.numberOfObjectRead++; - } catch (IOException e) { - LOGGER.error("Problme while reading the madkey fom sort temp file"); - throw new CarbonSortKeyAndGroupByException("Problem while reading the sort temp file ", e); - } - - //return out row - return holder; - } - /** * Reads row from file * @return Object[] * @throws CarbonSortKeyAndGroupByException */ - private Object[] getRowFromStreamWithOutKettle() throws CarbonSortKeyAndGroupByException { + private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException { // create new row of size 3 (1 for dims , 1 for high card , 1 for measures) Object[] holder = new Object[3]; @@ -480,60 +402,6 @@ public int getEntryCount() { } @Override public int compareTo(SortTempFileChunkHolder other) { - if (useKettle) { - return compareWithKettle(other); - - } else { - return compareWithOutKettle(other); - } - } - - // TODO Remove after kettle flow is removed. - private int compareWithKettle(SortTempFileChunkHolder other) { - int diff = 0; - - int normalIndex = 0; - int noDictionaryindex = 0; - - for (boolean isNoDictionary : isNoDictionaryDimensionColumn) { - - if (isNoDictionary) { - byte[] byteArr1 = (byte[]) returnRow[IgnoreDictionary.BYTE_ARRAY_INDEX_IN_ROW.getIndex()]; - - ByteBuffer buff1 = ByteBuffer.wrap(byteArr1); - - // extract a high card dims from complete byte[]. - NonDictionaryUtil - .extractSingleHighCardDims(byteArr1, noDictionaryindex, noDictionaryCount, buff1); - - byte[] byteArr2 = - (byte[]) other.returnRow[IgnoreDictionary.BYTE_ARRAY_INDEX_IN_ROW.getIndex()]; - - ByteBuffer buff2 = ByteBuffer.wrap(byteArr2); - - // extract a high card dims from complete byte[]. - NonDictionaryUtil - .extractSingleHighCardDims(byteArr2, noDictionaryindex, noDictionaryCount, buff2); - - int difference = UnsafeComparer.INSTANCE.compareTo(buff1, buff2); - if (difference != 0) { - return difference; - } - noDictionaryindex++; - } else { - int dimFieldA = NonDictionaryUtil.getDimension(normalIndex, returnRow); - int dimFieldB = NonDictionaryUtil.getDimension(normalIndex, other.returnRow); - diff = dimFieldA - dimFieldB; - if (diff != 0) { - return diff; - } - normalIndex++; - } - } - return diff; - } - - private int compareWithOutKettle(SortTempFileChunkHolder other) { int diff = 0; int index = 0; int noDictionaryIndex = 0; diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java deleted file mode 100644 index 03e8b25e42f..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java +++ /dev/null @@ -1,283 +0,0 @@ -/* - * 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.sortandgroupby.sortdatastep; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.common.logging.impl.StandardLogService; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory; -import org.apache.carbondata.processing.schema.metadata.SortObserver; -import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException; -import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows; -import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger; -import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters; -import org.apache.carbondata.processing.util.NonDictionaryUtil; - -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStep; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; - -public class SortKeyStep extends BaseStep { - - /** - * LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(SortKeyStep.class.getName()); - - /** - * CarbonSortKeyAndGroupByStepData - */ - private SortKeyStepData data; - - /** - * CarbonSortKeyAndGroupByStepMeta - */ - private SortKeyStepMeta meta; - - /** - * carbonSortKeys - */ - private SortDataRows sortDataRows; - - /** - * intermediateFileMerger - */ - private SortIntermediateFileMerger intermediateFileMerger; - - /** - * rowCounter - */ - private long readCounter; - - /** - * writeCounter - */ - private long writeCounter; - - /** - * logCounter - */ - private int logCounter; - - /** - * observer - */ - private SortObserver observer; - - /** - * To determine whether the column is dictionary or not. - */ - private boolean[] noDictionaryColMaping; - - /** - * CarbonSortKeyAndGroupByStep Constructor - * - * @param stepMeta - * @param stepDataInterface - * @param copyNr - * @param transMeta - * @param trans - */ - public SortKeyStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - super(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - /** - * Perform the equivalent of processing one row. Typically this means - * reading a row from input (getRow()) and passing a row to output - * (putRow)). - * - * @param smi The steps metadata to work with - * @param sdi The steps temporary working data to work with (database - * connections, result sets, caches, temporary variables, etc.) - * @return false if no more rows can be processed or an error occurred. - * @throws KettleException - */ - public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException { - // get step meta - this.meta = ((SortKeyStepMeta) smi); - StandardLogService.setThreadName(meta.getPartitionID(), null); - // get step data - this.data = ((SortKeyStepData) sdi); - - // get row - Object[] row = getRow(); - - // create sort observer - this.observer = new SortObserver(); - - // if row is null then this step can start processing the data - if (row == null) { - return processRowToNextStep(); - } - - // check if all records are null than send empty row to next step - else if (NonDictionaryUtil.checkAllValuesForNull(row)) { - // create empty row out size - int outSize = Integer.parseInt(meta.getOutputRowSize()); - - Object[] outRow = new Object[outSize]; - - // clone out row meta - this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone()); - - // get all fields - this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this); - - LOGGER.info("Record Procerssed For table: " + meta.getTabelName()); - LOGGER.info("Record Form Previous Step was null"); - String logMessage = "Summary: Carbon Sort Key Step: Read: " + 1 + ": Write: " + 1; - LOGGER.info(logMessage); - - putRow(data.getOutputRowMeta(), outRow); - setOutputDone(); - return false; - } - - // if first - if (first) { - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordSortRowsStepTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - first = false; - - // clone out row meta - this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone()); - - // get all fields - this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this); - - this.meta.setNoDictionaryCount( - NonDictionaryUtil.extractNoDictionaryCount(meta.getNoDictionaryDims())); - - this.noDictionaryColMaping = - NonDictionaryUtil.convertStringToBooleanArr(meta.getNoDictionaryDimsMapping()); - SortParameters parameters = - SortParameters.createSortParameters(meta.getDatabaseName(), meta.getTabelName(), - meta.getDimensionCount(), meta.getComplexDimensionCount(), meta.getMeasureCount(), - meta.getNoDictionaryCount(), meta.getPartitionID(), - meta.getSegmentId() + "", meta.getTaskNo(), this.noDictionaryColMaping); - intermediateFileMerger = new SortIntermediateFileMerger(parameters); - this.sortDataRows = new SortDataRows(parameters, intermediateFileMerger); - try { - // initialize sort - this.sortDataRows.initialize(); - } catch (CarbonSortKeyAndGroupByException e) { - throw new KettleException(e); - } - - this.logCounter = Integer.parseInt(CarbonProperties.getInstance() - .getProperty(CarbonCommonConstants.DATA_LOAD_LOG_COUNTER, - CarbonCommonConstants.DATA_LOAD_LOG_COUNTER_DEFAULT_COUNTER)); - } - - readCounter++; - if (readCounter % logCounter == 0) { - LOGGER.info("Record Procerssed For table: " + meta.getTabelName()); - String logMessage = "Carbon Sort Key Step: Record Read: " + readCounter; - LOGGER.info(logMessage); - } - - try { - // add row - this.sortDataRows.addRow(row); - writeCounter++; - } catch (Throwable e) { - LOGGER.error(e); - throw new KettleException(e); - } - - return true; - } - - /** - * Below method will be used to process data to next step - * - * @return false is finished - * @throws KettleException - */ - private boolean processRowToNextStep() throws KettleException { - if (null == this.sortDataRows) { - LOGGER.info("Record Processed For table: " + meta.getTabelName()); - LOGGER.info("Number of Records was Zero"); - String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0; - LOGGER.info(logMessage); - putRow(data.getOutputRowMeta(), new Object[0]); - setOutputDone(); - return false; - } - - try { - // start sorting - this.sortDataRows.startSorting(); - this.intermediateFileMerger.finish(); - - // check any more rows are present - LOGGER.info("Record Processed For table: " + meta.getTabelName()); - String logMessage = - "Summary: Carbon Sort Key Step: Read: " + readCounter + ": Write: " + writeCounter; - LOGGER.info(logMessage); - putRow(data.getOutputRowMeta(), new Object[0]); - setOutputDone(); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordSortRowsStepTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValuesTotalTime( - meta.getPartitionID(), System.currentTimeMillis()); - return false; - } catch (CarbonSortKeyAndGroupByException e) { - throw new KettleException(e); - } - - } - - /** - * Initialize and do work where other steps need to wait for... - * - * @param smi The metadata to work with - * @param sdi The data to initialize - * @return step initialize or not - */ - public boolean init(StepMetaInterface smi, StepDataInterface sdi) { - this.meta = ((SortKeyStepMeta) smi); - this.data = ((SortKeyStepData) sdi); - return super.init(smi, sdi); - } - - /** - * Dispose of this step: close files, empty logs, etc. - * - * @param smi The metadata to work with - * @param sdi The data to dispose of - */ - public void dispose(StepMetaInterface smi, StepDataInterface sdi) { - this.meta = ((SortKeyStepMeta) smi); - this.data = ((SortKeyStepData) sdi); - this.sortDataRows = null; - super.dispose(smi, sdi); - this.meta = null; - this.data = null; - } -} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java deleted file mode 100644 index 1a9859fc6f9..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.sortandgroupby.sortdatastep; - -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.step.BaseStepData; -import org.pentaho.di.trans.step.StepDataInterface; - -public class SortKeyStepData extends BaseStepData implements StepDataInterface { - - /** - * outputRowMeta - */ - private RowMetaInterface outputRowMeta; - - /** - * rowMeta - */ - private RowMetaInterface rowMeta; - - public RowMetaInterface getOutputRowMeta() { - return outputRowMeta; - } - - public void setOutputRowMeta(RowMetaInterface outputRowMeta) { - this.outputRowMeta = outputRowMeta; - } - - public RowMetaInterface getRowMeta() { - return rowMeta; - } - - public void setRowMeta(RowMetaInterface rowMeta) { - this.rowMeta = rowMeta; - } -} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java deleted file mode 100644 index 2b5481267d9..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java +++ /dev/null @@ -1,490 +0,0 @@ -/* - * 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.sortandgroupby.sortdatastep; - -import java.util.List; -import java.util.Map; - -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; - -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Counter; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.exception.KettleXMLException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.xml.XMLHandler; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.repository.ObjectId; -import org.pentaho.di.repository.Repository; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStepMeta; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.w3c.dom.Node; - -public class SortKeyStepMeta extends BaseStepMeta implements StepMetaInterface { - /** - * PKG - */ - private static final Class PKG = SortKeyStepMeta.class; - - /** - * tabelName - */ - private String tabelName; - - /** - * outputRowSize - */ - private String outputRowSize; - - /** - * tableName - */ - private String tableName; - - /** - * databaseName - */ - private String databaseName; - - /** - * Dimension Count - */ - private String dimensionCount; - - /** - * ComplexTypes Count - */ - private String complexDimensionCount; - - /** - * Dimension Count - */ - private int noDictionaryCount; - - /** - * measureCount - */ - private String measureCount; - - private String factDimLensString; - - /** - * isUpdateMemberRequest - */ - private String updateMemberRequest; - - private String measureDataType; - - private String noDictionaryDims; - /** - * partitionID - */ - private String partitionID; - /** - * Id of the load folder - */ - private String segmentId; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - /** - * To determine the column whether is dictionary or not. - */ - private String noDictionaryDimsMapping; - - /** - * set the default value for all the properties - */ - @Override public void setDefault() { - this.tabelName = ""; - factDimLensString = ""; - outputRowSize = ""; - databaseName = ""; - noDictionaryDims = ""; - noDictionaryDimsMapping = ""; - tableName = ""; - dimensionCount = ""; - complexDimensionCount = ""; - measureCount = ""; - updateMemberRequest = ""; - measureDataType = ""; - partitionID = ""; - segmentId = ""; - taskNo = ""; - } - - /** - * Get the XML that represents the values in this step - * - * @return the XML that represents the metadata in this step - * @throws KettleException in case there is a conversion or XML encoding error - */ - public String getXML() { - StringBuilder retval = new StringBuilder(150); - retval.append(" ").append(XMLHandler.addTagValue("TableName", this.tabelName)); - retval.append(" ").append(XMLHandler.addTagValue("factDimLensString", factDimLensString)); - retval.append(" ").append(XMLHandler.addTagValue("outputRowSize", this.outputRowSize)); - retval.append(" ").append(XMLHandler.addTagValue("tableName", this.tableName)); - retval.append(" ").append(XMLHandler.addTagValue("databaseName", this.databaseName)); - retval.append(" ").append(XMLHandler.addTagValue("dimensionCount", this.dimensionCount)); - retval.append(" ").append(XMLHandler.addTagValue("noDictionaryDims", this.noDictionaryDims)); - retval.append(" ") - .append(XMLHandler.addTagValue("noDictionaryDimsMapping", this.noDictionaryDimsMapping)); - retval.append(" ") - .append(XMLHandler.addTagValue("complexDimensionCount", this.complexDimensionCount)); - retval.append(" ").append(XMLHandler.addTagValue("measureCount", this.measureCount)); - retval.append(" ") - .append(XMLHandler.addTagValue("isUpdateMemberRequest", this.updateMemberRequest)); - retval.append(" ").append(XMLHandler.addTagValue("measureDataType", measureDataType)); - retval.append(" ").append(XMLHandler.addTagValue("partitionID", partitionID)); - retval.append(" ").append(XMLHandler.addTagValue("segmentId", segmentId)); - retval.append(" ").append(XMLHandler.addTagValue("taskNo", taskNo)); - return retval.toString(); - } - - /** - * Load the values for this step from an XML Node - * - * @param stepnode the Node to get the info from - * @param databases The available list of databases to reference to - * @param counters Counters to reference. - * @throws KettleXMLException When an unexpected XML error occurred. (malformed etc.) - */ - public void loadXML(Node stepnode, List databases, Map counters) - throws KettleXMLException { - try { - this.tabelName = XMLHandler.getTagValue(stepnode, "TableName"); - this.outputRowSize = XMLHandler.getTagValue(stepnode, "outputRowSize"); - this.factDimLensString = XMLHandler.getTagValue(stepnode, "factDimLensString"); - this.tableName = XMLHandler.getTagValue(stepnode, "tableName"); - this.databaseName = XMLHandler.getTagValue(stepnode, "databaseName"); - this.dimensionCount = XMLHandler.getTagValue(stepnode, "dimensionCount"); - this.noDictionaryDims = XMLHandler.getTagValue(stepnode, "noDictionaryDims"); - this.noDictionaryDimsMapping = XMLHandler.getTagValue(stepnode, "noDictionaryDimsMapping"); - this.complexDimensionCount = XMLHandler.getTagValue(stepnode, "complexDimensionCount"); - this.measureCount = XMLHandler.getTagValue(stepnode, "measureCount"); - this.updateMemberRequest = XMLHandler.getTagValue(stepnode, "isUpdateMemberRequest"); - this.measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType"); - this.partitionID = XMLHandler.getTagValue(stepnode, "partitionID"); - this.segmentId = XMLHandler.getTagValue(stepnode, "segmentId"); - this.taskNo = XMLHandler.getTagValue(stepnode, "taskNo"); - } catch (Exception e) { - throw new KettleXMLException("Unable to read step info from XML node", e); - } - } - - /** - * Save the steps data into a Kettle repository - * - * @param rep The Kettle repository to save to - * @param idTransformation The transformation ID - * @param idStep The step ID - * @throws KettleException When an unexpected error occurred (database, network, etc) - */ - public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep) - throws KettleException { - try { - rep.saveStepAttribute(idTransformation, idStep, "TableName", this.tabelName); - - rep.saveStepAttribute(idTransformation, idStep, "factDimLensString", factDimLensString); - rep.saveStepAttribute(idTransformation, idStep, "outputRowSize", this.outputRowSize); - rep.saveStepAttribute(idTransformation, idStep, "tableName", this.tableName); - rep.saveStepAttribute(idTransformation, idStep, "databaseName", this.databaseName); - rep.saveStepAttribute(idTransformation, idStep, "dimensionCount", this.dimensionCount); - rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDims", this.noDictionaryDims); - rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDimsMapping", - this.noDictionaryDimsMapping); - rep.saveStepAttribute(idTransformation, idStep, "complexDimensionCount", - this.complexDimensionCount); - rep.saveStepAttribute(idTransformation, idStep, "measureCount", this.measureCount); - rep.saveStepAttribute(idTransformation, idStep, "isUpdateMemberRequest", - this.updateMemberRequest); - rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType); - rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID); - rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId); - rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo); - } catch (Exception e) { - throw new KettleException(BaseMessages - .getString(PKG, "TemplateStep.Exception.UnableToSaveStepInfoToRepository", new String[0]) - + idStep, e); - } - } - - /** - * Read the steps information from a Kettle repository - * - * @param rep The repository to read from - * @param idStep The step ID - * @param databases The databases to reference - * @param counters The counters to reference - * @throws KettleException When an unexpected error occurred (database, network, etc) - */ - public void readRep(Repository rep, ObjectId idStep, List databases, - Map counters) throws KettleException { - try { - this.tabelName = rep.getStepAttributeString(idStep, "TableName"); - this.outputRowSize = rep.getStepAttributeString(idStep, "outputRowSize"); - this.databaseName = rep.getStepAttributeString(idStep, "databaseName"); - this.tableName = rep.getStepAttributeString(idStep, "tableName"); - this.dimensionCount = rep.getStepAttributeString(idStep, "dimensionCount"); - this.noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDims"); - this.noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDimsMapping"); - this.complexDimensionCount = rep.getStepAttributeString(idStep, "complexDimensionCount"); - this.measureCount = rep.getStepAttributeString(idStep, "measureCount"); - this.updateMemberRequest = rep.getStepAttributeString(idStep, "isUpdateMemberRequest"); - this.measureDataType = rep.getStepAttributeString(idStep, "measureDataType"); - this.partitionID = rep.getStepAttributeString(idStep, "partitionID"); - this.segmentId = rep.getStepAttributeString(idStep, "segmentId"); - this.taskNo = rep.getStepAttributeString(idStep, "taskNo"); - } catch (Exception ex) { - throw new KettleException(BaseMessages - .getString(PKG, "CarbonDataWriterStepMeta.Exception.UnexpectedErrorInReadingStepInfo", - new String[0]), ex); - } - } - - /** - * Checks the settings of this step and puts the findings in a remarks List. - * - * @param remarks The list to put the remarks in @see - * org.pentaho.di.core.CheckResult - * @param stepMeta The stepMeta to help checking - * @param prev The fields coming from the previous step - * @param input The input step names - * @param output The output step names - * @param info The fields that are used as information by the step - */ - public void check(List remarks, TransMeta transMeta, StepMeta stepMeta, - RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) { - CarbonDataProcessorUtil.checkResult(remarks, stepMeta, input); - } - - /** - * Get the executing step, needed by Trans to launch a step. - * - * @param stepMeta The step info - * @param stepDataInterface the step data interface linked to this step. Here the step can - * store temporary data, database connections, etc. - * @param copyNr The copy nr to get - * @param transMeta The transformation info - * @param trans The launching transformation - */ - public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr, - TransMeta transMeta, Trans trans) { - return new SortKeyStep(stepMeta, stepDataInterface, copyNr, transMeta, trans); - } - - /** - * Get a new instance of the appropriate data class. This data class - * implements the StepDataInterface. It basically contains the persisting - * data that needs to live on, even if a worker thread is terminated. - * - * @return The appropriate StepDataInterface class. - */ - public StepDataInterface getStepData() { - return new SortKeyStepData(); - } - - /** - * Below method will be used to get the out row size - * - * @return outputRowSize - */ - public String getOutputRowSize() { - return outputRowSize; - } - - /** - * below mthod will be used to set the out row size - * - * @param outputRowSize - */ - public void setOutputRowSize(String outputRowSize) { - this.outputRowSize = outputRowSize; - } - - /** - * This method will return the table name - * - * @return tabelName - */ - - public String getTabelName() { - return this.tabelName; - } - - /** - * This method will set the table name - * - * @param tabelName - */ - public void setTabelName(String tabelName) { - this.tabelName = tabelName; - } - - /** - * @param tableName the tableName to set - */ - public void setTableName(String tableName) { - this.tableName = tableName; - } - - /** - * @return the databaseName - */ - public String getDatabaseName() { - return databaseName; - } - - /** - * @param databaseName the databaseName to set - */ - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - /** - * @return the dimensionCount - */ - public int getDimensionCount() { - return Integer.parseInt(dimensionCount); - } - - public void setDimensionCount(String dimensionCount) { - this.dimensionCount = dimensionCount; - } - - /** - * @return the complexDimensionCount - */ - public int getComplexDimensionCount() { - return Integer.parseInt(complexDimensionCount); - } - - public void setComplexDimensionCount(String complexDimensionCount) { - this.complexDimensionCount = complexDimensionCount; - } - - /** - * @return the measureCount - */ - public int getMeasureCount() { - return Integer.parseInt(measureCount); - } - - /** - * @param measureCount the measureCount to set - */ - public void setMeasureCount(String measureCount) { - this.measureCount = measureCount; - } - - /** - * @param isUpdateMemberRequest the isUpdateMemberRequest to set - */ - public void setIsUpdateMemberRequest(String isUpdateMemberRequest) { - this.updateMemberRequest = isUpdateMemberRequest; - } - - public void setMeasureDataType(String measureDataType) { - this.measureDataType = measureDataType; - } - - public String getNoDictionaryDims() { - return noDictionaryDims; - } - - public void setNoDictionaryDims(String noDictionaryDims) { - this.noDictionaryDims = noDictionaryDims; - } - - /** - * @return the noDictionaryCount - */ - public int getNoDictionaryCount() { - return noDictionaryCount; - } - - /** - * @param noDictionaryCount the noDictionaryCount to set - */ - public void setNoDictionaryCount(int noDictionaryCount) { - this.noDictionaryCount = noDictionaryCount; - } - - /** - * @return partitionId - */ - public String getPartitionID() { - return partitionID; - } - - /** - * @param partitionID - */ - public void setPartitionID(String partitionID) { - this.partitionID = partitionID; - } - - /** - * return segmentId - * - * @return - */ - public String getSegmentId() { - return segmentId; - } - - /** - * set segment Id - * - * @param segmentId - */ - public void setSegmentId(String segmentId) { - this.segmentId = segmentId; - } - - /** - * @param taskNo - */ - public void setTaskNo(String taskNo) { - this.taskNo = taskNo; - } - - /** - * @return - */ - public String getTaskNo() { - return taskNo; - } - - public String getNoDictionaryDimsMapping() { - return noDictionaryDimsMapping; - } - - public void setNoDictionaryDimsMapping(String noDictionaryDimsMapping) { - this.noDictionaryDimsMapping = noDictionaryDimsMapping; - } -} \ No newline at end of file diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java index 3806c55ee39..2affa038cc8 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java @@ -65,13 +65,13 @@ import org.apache.carbondata.core.util.NodeHolder; import org.apache.carbondata.core.util.ValueCompressionUtil; import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.mdkeygen.file.FileManager; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; import org.apache.carbondata.processing.store.colgroup.ColGroupBlockStorage; import org.apache.carbondata.processing.store.colgroup.ColGroupDataHolder; import org.apache.carbondata.processing.store.colgroup.ColGroupMinMax; import org.apache.carbondata.processing.store.colgroup.ColumnDataHolder; import org.apache.carbondata.processing.store.colgroup.DataHolder; +import org.apache.carbondata.processing.store.file.FileManager; +import org.apache.carbondata.processing.store.file.IFileManagerComposite; import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo; import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter; import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException; @@ -256,8 +256,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler { */ private boolean compactionFlow; - private boolean useKettle; - private int bucketNumber; private long schemaUpdatedTimeStamp; @@ -279,7 +277,6 @@ public CarbonFactDataHandlerColumnar(CarbonFactDataHandlerModel carbonFactDataHa this.dimensionCount = carbonFactDataHandlerModel.getDimensionCount(); this.complexIndexMap = carbonFactDataHandlerModel.getComplexIndexMap(); this.primitiveDimLens = carbonFactDataHandlerModel.getPrimitiveDimLens(); - this.useKettle = carbonFactDataHandlerModel.isUseKettle(); this.isAggKeyBlock = Boolean.parseBoolean(CarbonProperties.getInstance() .getProperty(CarbonCommonConstants.AGGREAGATE_COLUMNAR_KEY_BLOCK, CarbonCommonConstants.AGGREAGATE_COLUMNAR_KEY_BLOCK_DEFAULTVALUE)); @@ -481,144 +478,7 @@ public void addDataToStore(Object[] row) throws CarbonDataWriterException { } } - // TODO remove after kettle flow is removed - private NodeHolder processDataRows(List dataRows) throws CarbonDataWriterException { - Object[] max = new Object[measureCount]; - Object[] min = new Object[measureCount]; - int[] decimal = new int[measureCount]; - Object[] uniqueValue = new Object[measureCount]; - // to store index of the measure columns which are null - BitSet[] nullValueIndexBitSet = getMeasureNullValueIndexBitSet(measureCount); - for (int i = 0; i < max.length; i++) { - if (type[i] == CarbonCommonConstants.BIG_INT_MEASURE) { - max[i] = Long.MIN_VALUE; - } else if (type[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) { - max[i] = -Double.MAX_VALUE; - } else if (type[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - max[i] = new BigDecimal(-Double.MAX_VALUE); - } else { - max[i] = 0.0; - } - } - for (int i = 0; i < min.length; i++) { - if (type[i] == CarbonCommonConstants.BIG_INT_MEASURE) { - min[i] = Long.MAX_VALUE; - uniqueValue[i] = Long.MIN_VALUE; - } else if (type[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) { - min[i] = Double.MAX_VALUE; - uniqueValue[i] = Double.MIN_VALUE; - } else if (type[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - min[i] = new BigDecimal(Double.MAX_VALUE); - uniqueValue[i] = new BigDecimal(Double.MIN_VALUE); - } else { - min[i] = 0.0; - uniqueValue[i] = 0.0; - } - } - for (int i = 0; i < decimal.length; i++) { - decimal[i] = 0; - } - - byte[] startKey = null; - byte[] endKey = null; - byte[] noDictStartKey = null; - byte[] noDictEndKey = null; - CarbonWriteDataHolder[] dataHolder = initialiseDataHolder(dataRows.size()); - CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolder(dataRows.size()); - CarbonWriteDataHolder noDictionaryKeyDataHolder = null; - if ((noDictionaryCount + complexColCount) > 0) { - noDictionaryKeyDataHolder = initialiseKeyBlockHolder(dataRows.size()); - } - - for (int count = 0; count < dataRows.size(); count++) { - Object[] row = dataRows.get(count); - byte[] mdKey = (byte[]) row[this.mdKeyIndex]; - byte[] noDictionaryKey = null; - if (noDictionaryCount > 0 || complexIndexMap.size() > 0) { - noDictionaryKey = (byte[]) row[this.mdKeyIndex - 1]; - } - ByteBuffer byteBuffer = null; - byte[] b = null; - if (count == 0) { - startKey = mdKey; - noDictStartKey = noDictionaryKey; - } - endKey = mdKey; - noDictEndKey = noDictionaryKey; - // add to key store - if (mdKey.length > 0) { - keyDataHolder.setWritableByteArrayValueByIndex(count, mdKey); - } - // for storing the byte [] for high card. - if (noDictionaryCount > 0 || complexIndexMap.size() > 0) { - noDictionaryKeyDataHolder.setWritableByteArrayValueByIndex(count, noDictionaryKey); - } - //Add all columns to keyDataHolder - keyDataHolder.setWritableByteArrayValueByIndex(count, this.mdKeyIndex, row); - // CHECKSTYLE:OFF Approval No:Approval-351 - for (int k = 0; k < otherMeasureIndex.length; k++) { - if (type[otherMeasureIndex[k]] == CarbonCommonConstants.BIG_INT_MEASURE) { - if (null == row[otherMeasureIndex[k]]) { - nullValueIndexBitSet[otherMeasureIndex[k]].set(count); - dataHolder[otherMeasureIndex[k]].setWritableLongValueByIndex(count, 0L); - } else { - dataHolder[otherMeasureIndex[k]] - .setWritableLongValueByIndex(count, row[otherMeasureIndex[k]]); - } - } else { - if (null == row[otherMeasureIndex[k]]) { - nullValueIndexBitSet[otherMeasureIndex[k]].set(count); - dataHolder[otherMeasureIndex[k]].setWritableDoubleValueByIndex(count, 0.0); - } else { - dataHolder[otherMeasureIndex[k]] - .setWritableDoubleValueByIndex(count, row[otherMeasureIndex[k]]); - } - } - } - calculateMaxMin(max, min, decimal, otherMeasureIndex, row); - for (int i = 0; i < customMeasureIndex.length; i++) { - if (null == row[customMeasureIndex[i]] - && type[customMeasureIndex[i]] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) { - BigDecimal val = BigDecimal.valueOf(0); - b = DataTypeUtil.bigDecimalToByte(val); - nullValueIndexBitSet[customMeasureIndex[i]].set(count); - } else { - if (this.compactionFlow) { - BigDecimal bigDecimal = ((Decimal) row[customMeasureIndex[i]]).toJavaBigDecimal(); - b = DataTypeUtil.bigDecimalToByte(bigDecimal); - } else { - b = (byte[]) row[customMeasureIndex[i]]; - } - } - byteBuffer = ByteBuffer.allocate(b.length + CarbonCommonConstants.INT_SIZE_IN_BYTE); - byteBuffer.putInt(b.length); - byteBuffer.put(b); - byteBuffer.flip(); - b = byteBuffer.array(); - dataHolder[customMeasureIndex[i]].setWritableByteArrayValueByIndex(count, b); - } - calculateMaxMin(max, min, decimal, customMeasureIndex, row); - } - calculateUniqueValue(min, uniqueValue); - byte[][] byteArrayValues = keyDataHolder.getByteArrayValues().clone(); - byte[][] noDictionaryValueHolder = null; - if ((noDictionaryCount + complexColCount) > 0) { - noDictionaryValueHolder = noDictionaryKeyDataHolder.getByteArrayValues(); - } - WriterCompressModel compressionModel = ValueCompressionUtil - .getWriterCompressModel(max, min, decimal, uniqueValue, type, new byte[max.length]); - byte[][] writableMeasureDataArray = - StoreFactory.createDataStore(compressionModel).getWritableMeasureDataArray(dataHolder) - .clone(); - NodeHolder nodeHolder = - getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey, - endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey, - nullValueIndexBitSet); - LOGGER.info("Number Of records processed: " + dataRows.size()); - return nodeHolder; - } - - private NodeHolder processDataRowsWithOutKettle(List dataRows) + private NodeHolder processDataRows(List dataRows) throws CarbonDataWriterException { Object[] max = new Object[measureCount]; Object[] min = new Object[measureCount]; @@ -661,7 +521,7 @@ private NodeHolder processDataRowsWithOutKettle(List dataRows) byte[][] noDictStartKey = null; byte[][] noDictEndKey = null; CarbonWriteDataHolder[] dataHolder = initialiseDataHolder(dataRows.size()); - CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolderWithOutKettle(dataRows.size()); + CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolder(dataRows.size()); CarbonWriteDataHolder noDictionaryKeyDataHolder = null; if ((noDictionaryCount + complexColCount) > 0) { noDictionaryKeyDataHolder = initialiseKeyBlockHolderForNonDictionary(dataRows.size()); @@ -756,142 +616,14 @@ private NodeHolder processDataRowsWithOutKettle(List dataRows) StoreFactory.createDataStore(compressionModel).getWritableMeasureDataArray(dataHolder) .clone(); NodeHolder nodeHolder = - getNodeHolderObjectWithOutKettle(writableMeasureDataArray, byteArrayValues, dataRows.size(), + getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey, endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey, nullValueIndexBitSet); LOGGER.info("Number Of records processed: " + dataRows.size()); return nodeHolder; } - // TODO remove after kettle flow is removed - private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues, - int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal, - WriterCompressModel compressionModel, byte[][] noDictionaryData, byte[] noDictionaryStartKey, - byte[] noDictionaryEndKey, BitSet[] nullValueIndexBitSet) throws CarbonDataWriterException { - byte[][][] noDictionaryColumnsData = null; - List> colsAndValues = new ArrayList>(); - int complexColCount = getComplexColsCount(); - - for (int i = 0; i < complexColCount; i++) { - colsAndValues.add(new ArrayList()); - } - int noOfColumn = colGrpModel.getNoOfColumnStore(); - DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length); - for (int i = 0; i < byteArrayValues.length; i++) { - byte[][] splitKey = columnarSplitter.splitKey(byteArrayValues[i]); - - for (int j = 0; j < splitKey.length; j++) { - dataHolders[j].addData(splitKey[j], i); - } - } - if (noDictionaryCount > 0 || complexIndexMap.size() > 0) { - noDictionaryColumnsData = new byte[noDictionaryCount][noDictionaryData.length][]; - for (int i = 0; i < noDictionaryData.length; i++) { - int complexColumnIndex = primitiveDimLens.length + noDictionaryCount; - byte[][] splitKey = NonDictionaryUtil - .splitNoDictionaryKey(noDictionaryData[i], noDictionaryCount + complexIndexMap.size()); - - int complexTypeIndex = 0; - for (int j = 0; j < splitKey.length; j++) { - //nodictionary Columns - if (j < noDictionaryCount) { - noDictionaryColumnsData[j][i] = splitKey[j]; - } - //complex types - else { - // Need to write columnar block from complex byte array - int index = complexColumnIndex - noDictionaryCount; - GenericDataType complexDataType = complexIndexMap.get(index); - complexColumnIndex++; - if (complexDataType != null) { - List> columnsArray = new ArrayList>(); - for (int k = 0; k < complexDataType.getColsCount(); k++) { - columnsArray.add(new ArrayList()); - } - - try { - ByteBuffer complexDataWithoutBitPacking = ByteBuffer.wrap(splitKey[j]); - byte[] complexTypeData = new byte[complexDataWithoutBitPacking.getShort()]; - complexDataWithoutBitPacking.get(complexTypeData); - - ByteBuffer byteArrayInput = ByteBuffer.wrap(complexTypeData); - ByteArrayOutputStream byteArrayOutput = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutput); - complexDataType - .parseAndBitPack(byteArrayInput, dataOutputStream, this.complexKeyGenerator); - complexDataType.getColumnarDataForComplexType(columnsArray, - ByteBuffer.wrap(byteArrayOutput.toByteArray())); - byteArrayOutput.close(); - } catch (IOException e) { - throw new CarbonDataWriterException( - "Problem while bit packing and writing complex datatype", e); - } catch (KeyGenException e) { - throw new CarbonDataWriterException( - "Problem while bit packing and writing complex datatype", e); - } - - for (ArrayList eachColumn : columnsArray) { - colsAndValues.get(complexTypeIndex++).addAll(eachColumn); - } - } else { - // This case not possible as ComplexType is the last columns - } - } - } - } - } - thread_pool_size = Integer.parseInt(CarbonProperties.getInstance() - .getProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT, - CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL)); - ExecutorService executorService = Executors.newFixedThreadPool(thread_pool_size); - List> submit = new ArrayList>( - primitiveDimLens.length + noDictionaryCount + complexColCount); - int i = 0; - int dictionaryColumnCount = -1; - int noDictionaryColumnCount = -1; - for (i = 0; i < dimensionType.length; i++) { - if (dimensionType[i]) { - dictionaryColumnCount++; - if (colGrpModel.isColumnar(dictionaryColumnCount)) { - submit.add(executorService.submit( - new BlockSortThread(i, dataHolders[dictionaryColumnCount].getData(), true, - isUseInvertedIndex[i]))); - } else { - submit.add( - executorService.submit(new ColGroupBlockStorage(dataHolders[dictionaryColumnCount]))); - } - } else { - submit.add(executorService.submit( - new BlockSortThread(i, noDictionaryColumnsData[++noDictionaryColumnCount], false, true, - true, isUseInvertedIndex[i]))); - } - } - for (int k = 0; k < complexColCount; k++) { - submit.add(executorService.submit(new BlockSortThread(i++, - colsAndValues.get(k).toArray(new byte[colsAndValues.get(k).size()][]), false, true))); - } - executorService.shutdown(); - try { - executorService.awaitTermination(1, TimeUnit.DAYS); - } catch (InterruptedException e) { - LOGGER.error(e, e.getMessage()); - } - IndexStorage[] blockStorage = - new IndexStorage[colGrpModel.getNoOfColumnStore() + noDictionaryCount + complexColCount]; - try { - for (int k = 0; k < blockStorage.length; k++) { - blockStorage[k] = submit.get(k).get(); - } - } catch (Exception e) { - LOGGER.error(e, e.getMessage()); - } - return this.dataWriter - .buildDataNodeHolder(blockStorage, dataHolderLocal, entryCountLocal, startkeyLocal, - endKeyLocal, compressionModel, noDictionaryStartKey, noDictionaryEndKey, - nullValueIndexBitSet); - } - - private NodeHolder getNodeHolderObjectWithOutKettle(byte[][] dataHolderLocal, + private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues, int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal, WriterCompressModel compressionModel, byte[][][] noDictionaryData, byte[][] noDictionaryStartKey, byte[][] noDictionaryEndKey, BitSet[] nullValueIndexBitSet) @@ -1343,16 +1075,9 @@ private int[] getBlockKeySizeWithComplexTypes(int[] primitiveBlockKeySize) { return blockKeySizeWithComplexTypes; } - // TODO Remove after kettle flow got removed. private CarbonWriteDataHolder initialiseKeyBlockHolder(int size) { CarbonWriteDataHolder keyDataHolder = new CarbonWriteDataHolder(); - keyDataHolder.initialiseByteArrayValues(size); - return keyDataHolder; - } - - private CarbonWriteDataHolder initialiseKeyBlockHolderWithOutKettle(int size) { - CarbonWriteDataHolder keyDataHolder = new CarbonWriteDataHolder(); - keyDataHolder.initialiseByteArrayValuesWithOutKettle(size); + keyDataHolder.initialiseByteArrayValuesForKey(size); return keyDataHolder; } @@ -1556,12 +1281,7 @@ private Producer(BlockletDataHolder blockletDataHolder, List dataRows, */ @Override public Void call() throws Exception { try { - NodeHolder nodeHolder; - if (useKettle) { - nodeHolder = processDataRows(dataRows); - } else { - nodeHolder = processDataRowsWithOutKettle(dataRows); - } + NodeHolder nodeHolder = processDataRows(dataRows); // insert the object in array according to sequence number int indexInNodeHolderArray = (sequenceNumber - 1) % numberOfCores; blockletDataHolder.put(nodeHolder, indexInNodeHolderArray); diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java index 59f2eb379e3..ffd23a2ba7a 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java @@ -171,11 +171,6 @@ public void setBlockSizeInMB(int blockSize) { */ private boolean isCompactionFlow; - /** - * To use kettle flow to load or not. - */ - private boolean useKettle = true; - private int bucketId = 0; private String segmentId; @@ -287,7 +282,6 @@ public static CarbonFactDataHandlerModel createCarbonFactDataHandlerModel( carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath); carbonFactDataHandlerModel.setIsUseInvertedIndex(isUseInvertedIndex); carbonFactDataHandlerModel.setBlockSizeInMB(carbonTable.getBlockSizeInMB()); - carbonFactDataHandlerModel.setUseKettle(false); if (noDictionaryCount > 0 || complexDimensionCount > 0) { carbonFactDataHandlerModel.setMdKeyIndex(measureCount + 1); } else { @@ -500,14 +494,6 @@ public void setWrapperColumnSchema(List wrapperColumnSchema) { this.wrapperColumnSchema = wrapperColumnSchema; } - public boolean isUseKettle() { - return useKettle; - } - - public void setUseKettle(boolean useKettle) { - this.useKettle = useKettle; - } - public int getBucketId() { return bucketId; } diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java index 8c94328b139..68f9bd57f5b 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java @@ -95,8 +95,6 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator { private char[] aggType; - private boolean useKettle; - /** * below code is to check whether dimension * is of no dictionary type or not @@ -105,7 +103,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator { public SingleThreadFinalSortFilesMerger(String tempFileLocation, String tableName, int dimensionCount, int complexDimensionCount, int measureCount, int noDictionaryCount, - char[] aggType, boolean[] isNoDictionaryColumn, boolean useKettle) { + char[] aggType, boolean[] isNoDictionaryColumn) { this.tempFileLocation = tempFileLocation; this.tableName = tableName; this.dimensionCount = dimensionCount; @@ -114,7 +112,6 @@ public SingleThreadFinalSortFilesMerger(String tempFileLocation, String tableNam this.aggType = aggType; this.noDictionaryCount = noDictionaryCount; this.isNoDictionaryColumn = isNoDictionaryColumn; - this.useKettle = useKettle; } /** @@ -183,8 +180,7 @@ private void startSorting(File[] files) throws CarbonDataWriterException { // create chunk holder SortTempFileChunkHolder sortTempFileChunkHolder = new SortTempFileChunkHolder(tempFile, dimensionCount, complexDimensionCount, - measureCount, fileBufferSize, noDictionaryCount, aggType, isNoDictionaryColumn, - useKettle); + measureCount, fileBufferSize, noDictionaryCount, aggType, isNoDictionaryColumn); // initialize sortTempFileChunkHolder.initialize(); diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileData.java b/processing/src/main/java/org/apache/carbondata/processing/store/file/FileData.java similarity index 69% rename from processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileData.java rename to processing/src/main/java/org/apache/carbondata/processing/store/file/FileData.java index 024a0b9cd7f..ddd9bf2ab6e 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileData.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/file/FileData.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.carbondata.processing.mdkeygen.file; +package org.apache.carbondata.processing.store.file; -import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV; public class FileData extends FileManager { @@ -29,7 +28,6 @@ public class FileData extends FileManager { /** * hierarchyValueWriter */ - private HierarchyValueWriterForCSV hierarchyValueWriter; public FileData(String fileName, String storePath) { this.fileName = fileName; @@ -50,23 +48,5 @@ public String getStorePath() { return storePath; } - /** - * get Hierarchy Value writer - * - * @return - */ - public HierarchyValueWriterForCSV getHierarchyValueWriter() { - return hierarchyValueWriter; - } - - /** - * Set Hierarchy Value Writer. - * - * @param hierarchyValueWriter - */ - public void setHierarchyValueWriter(HierarchyValueWriterForCSV hierarchyValueWriter) { - this.hierarchyValueWriter = hierarchyValueWriter; - } - } diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java b/processing/src/main/java/org/apache/carbondata/processing/store/file/FileManager.java similarity index 96% rename from processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java rename to processing/src/main/java/org/apache/carbondata/processing/store/file/FileManager.java index c86725154f2..cfa3a66f5ae 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/file/FileManager.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.processing.mdkeygen.file; +package org.apache.carbondata.processing.store.file; import java.util.ArrayList; import java.util.List; diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/IFileManagerComposite.java b/processing/src/main/java/org/apache/carbondata/processing/store/file/IFileManagerComposite.java similarity index 96% rename from processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/IFileManagerComposite.java rename to processing/src/main/java/org/apache/carbondata/processing/store/file/IFileManagerComposite.java index 15e06a5acd2..6691772f8dc 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/IFileManagerComposite.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/file/IFileManagerComposite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.processing.mdkeygen.file; +package org.apache.carbondata.processing.store.file; public interface IFileManagerComposite { /** diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java index a73b356c712..cda907c97bc 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java @@ -66,7 +66,7 @@ import org.apache.carbondata.format.BlockIndex; import org.apache.carbondata.format.BlockletInfo3; import org.apache.carbondata.format.IndexHeader; -import org.apache.carbondata.processing.mdkeygen.file.FileData; +import org.apache.carbondata.processing.store.file.FileData; import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException; import org.apache.commons.lang3.ArrayUtils; diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java index 9ed0baa011b..ce53ec87897 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java @@ -20,8 +20,8 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; import org.apache.carbondata.processing.store.CarbonDataFileAttributes; +import org.apache.carbondata.processing.store.file.IFileManagerComposite; /** * Value object for writing the data diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java deleted file mode 100644 index 86b63df174b..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java +++ /dev/null @@ -1,495 +0,0 @@ -/* - * 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.surrogatekeysgenerator.csvbased; - -import java.sql.Connection; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantReadWriteLock; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.cache.dictionary.Dictionary; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; -import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; -import org.apache.carbondata.processing.schema.metadata.ArrayWrapper; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails; -import org.apache.carbondata.processing.schema.metadata.ColumnsInfo; - -import org.pentaho.di.core.exception.KettleException; - -public abstract class CarbonCSVBasedDimSurrogateKeyGen { - - private static final LogService LOGGER = - LogServiceFactory.getLogService(CarbonCSVBasedDimSurrogateKeyGen.class.getName()); - /** - * max - */ - protected int[] max; - /** - * connection - */ - protected Connection connection; - /** - * hierInsertFileNames - */ - protected Map hierInsertFileNames; - /** - * dimInsertFileNames - */ - protected String[] dimInsertFileNames; - /** - * columnsInfo - */ - protected ColumnsInfo columnsInfo; - /** - * primary key max surrogate key map - */ - protected Map primaryKeysMaxSurroagetMap; - /** - * Measure max surrogate key map - */ - protected Map measureMaxSurroagetMap; - /** - * File manager - */ - protected IFileManagerComposite fileManager; - - /** - * Cache should be map only. because, multiple levels can map to same - * database column. This case duplicate storage should be avoided. - */ - private Map dictionaryCaches; - /** - * Year Cache - */ - private Map> timeDimCache; - /** - * dimsFiles - */ - private String[] dimsFiles; - /** - * timeDimMax - */ - private int[] timDimMax; - /** - * hierCache - */ - private Map> hierCache = - new HashMap>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - /** - * - */ - private Map> hierCacheReverse = - new HashMap>( - CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - /** - * dimension ordinal to dimension mapping - */ - private CarbonDimension[] dimensionOrdinalToDimensionMapping; - /** - * rwLock2 - */ - private ReentrantReadWriteLock rwLock2 = new ReentrantReadWriteLock(); - /** - * wLock2 - */ - protected Lock wLock2 = rwLock2.writeLock(); - /** - * Store Folder Name with Load number. - */ - private String storeFolderWithLoadNumber; - - /** - * @param columnsInfo ColumnsInfo With all the required details for surrogate key generation and - * hierarchy entries. - */ - public CarbonCSVBasedDimSurrogateKeyGen(ColumnsInfo columnsInfo) { - this.columnsInfo = columnsInfo; - - setDimensionTables(columnsInfo.getDimColNames()); - setHierFileNames(columnsInfo.getHierTables()); - } - - /** - * @param tuple The string value whose surrogate key will be gennerated. - * @param tabColumnName The K of dictionaryCaches Map, for example "tablename_columnname" - */ - public Integer generateSurrogateKeys(String tuple, String tabColumnName) throws KettleException { - Integer key = null; - Dictionary dicCache = dictionaryCaches.get(tabColumnName); - key = dicCache.getSurrogateKey(tuple); - return key; - } - - /** - * @param tuple The string value whose surrogate key will be gennerated. - * @param tabColumnName The K of dictionaryCaches Map, for example "tablename_columnname" - */ - public Integer generateSurrogateKeys(String tuple, String tabColumnName, String columnId) - throws KettleException { - Integer key = null; - Dictionary dicCache = dictionaryCaches.get(tabColumnName); - if (null == dicCache) { - ColumnSchemaDetails columnSchemaDetails = - this.columnsInfo.getColumnSchemaDetailsWrapper().get(columnId); - if (columnSchemaDetails.isDirectDictionary()) { - DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory - .getDirectDictionaryGenerator(columnSchemaDetails.getColumnType()); - key = directDictionaryGenerator.generateDirectSurrogateKey(tuple); - } - } else { - key = dicCache.getSurrogateKey(tuple); - } - return key; - } - - - public Integer generateSurrogateKeysForTimeDims(String tuple, String columnName, int index, - Object[] props) throws KettleException { - Integer key = null; - Dictionary dicCache = dictionaryCaches.get(columnName); - key = dicCache.getSurrogateKey(tuple); - if (key == null) { - if (timDimMax[index] >= columnsInfo.getMaxKeys()[index]) { - if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(tuple)) { - tuple = null; - } - LOGGER.error("Invalid cardinality. Key size exceeded cardinality for: " + columnsInfo - .getDimColNames()[index] + ": MemberValue: " + tuple); - return -1; - } - timDimMax[index]++; - Map timeCache = timeDimCache.get(columnName); - // Extract properties from tuple - // Need to create a new surrogate key. - key = getSurrogateFromStore(tuple, index, props); - if (null != timeCache) { - timeCache.put(tuple, key); - } - } else { - return updateSurrogateToStore(tuple, columnName, index, key, props); - } - return key; - } - - public void checkNormalizedHierExists(int[] val, String hier, - HierarchyValueWriterForCSV hierWriter) throws KettleException { - Map cache = hierCacheReverse.get(hier); - - ArrayWrapper wrapper = new ArrayWrapper(val); - Integer hCache = cache.get(wrapper); - if (hCache != null) { - return; - } else { - wLock2.lock(); - try { - getNormalizedHierFromStore(val, hier, 1, hierWriter); - // Store in cache - cache.put(wrapper, 1); - } finally { - wLock2.unlock(); - } - } - } - - public void close() throws Exception { - if (null != connection) { - connection.close(); - } - } - - /** - * Search entry and insert if not found in store. - * - * @param val - * @param hier - * @return - * @throws KeyGenException - * @throws KettleException - */ - protected abstract byte[] getNormalizedHierFromStore(int[] val, String hier, int primaryKey, - HierarchyValueWriterForCSV hierWriter) throws KettleException; - - /** - * Search entry and insert if not found in store. - * - * @param value - * @param index - * @param properties - Ordinal column, name column and all other properties - * @return - * @throws KettleException - */ - protected abstract int getSurrogateFromStore(String value, int index, Object[] properties) - throws KettleException; - - /** - * Search entry and insert if not found in store. - * - * @param value - * @param columnName - * @param index - * @param properties - Ordinal column, name column and all other properties - * @return - * @throws KettleException - */ - protected abstract int updateSurrogateToStore(String value, String columnName, int index, int key, - Object[] properties) throws KettleException; - - /** - * generate the surroagate key for the measure values. - * - * @return - * @throws KettleException - */ - public abstract int getSurrogateForMeasure(String tuple, String columnName) - throws KettleException; - - private Map getHCache(String hName) { - Map hCache = hierCache.get(hName); - if (hCache == null) { - hCache = new HashMap(); - hierCache.put(hName, hCache); - } - - return hCache; - } - - private Map getHCacheReverse(String hName) { - Map hCache = hierCacheReverse.get(hName); - if (hCache == null) { - hCache = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - hierCacheReverse.put(hName, hCache); - } - - return hCache; - } - - private void setHierFileNames(Set set) { - hierInsertFileNames = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - for (String s : set) { - hierInsertFileNames.put(s, s + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION); - - // fix hierStream is null issue - getHCache(s); - getHCacheReverse(s); - } - } - - private void setDimensionTables(String[] dimeFileNames) { - int noOfPrimitiveDims = 0; - List dimFilesForPrimitives = new ArrayList(); - List isDirectDictionary = new ArrayList(); - dictionaryCaches = new ConcurrentHashMap(); - for (int i = 0; i < dimeFileNames.length; i++) { - GenericDataType complexType = columnsInfo.getComplexTypesMap() - .get(dimeFileNames[i].substring(columnsInfo.getTableName().length() + 1)); - if (complexType != null) { - List primitiveChild = new ArrayList(); - complexType.getAllPrimitiveChildren(primitiveChild); - for (GenericDataType eachPrimitive : primitiveChild) { - dimFilesForPrimitives.add( - columnsInfo.getTableName() + CarbonCommonConstants.UNDERSCORE + eachPrimitive - .getName()); - eachPrimitive.setSurrogateIndex(noOfPrimitiveDims); - noOfPrimitiveDims++; - ColumnSchemaDetails columnSchemaDetails = - columnsInfo.getColumnSchemaDetailsWrapper().get(eachPrimitive.getColumnId()); - if (columnSchemaDetails.isDirectDictionary()) { - isDirectDictionary.add(true); - } else { - isDirectDictionary.add(false); - } - } - } else { - dimFilesForPrimitives.add(dimeFileNames[i]); - noOfPrimitiveDims++; - isDirectDictionary.add(false); - } - } - max = new int[noOfPrimitiveDims]; - for (int i = 0; i < isDirectDictionary.size(); i++) { - if (isDirectDictionary.get(i)) { - max[i] = Integer.MAX_VALUE; - } - } - this.dimsFiles = dimFilesForPrimitives.toArray(new String[dimFilesForPrimitives.size()]); - - createRespectiveDimFilesForDimTables(); - } - - private void createRespectiveDimFilesForDimTables() { - int dimCount = this.dimsFiles.length; - dimInsertFileNames = new String[dimCount]; - System.arraycopy(dimsFiles, 0, dimInsertFileNames, 0, dimCount); - } - - /** - * isCacheFilled - * - * @param columnNames - * @return boolean - */ - public abstract boolean isCacheFilled(String[] columnNames); - - /** - * @return Returns the storeFolderWithLoadNumber. - */ - public String getStoreFolderWithLoadNumber() { - return storeFolderWithLoadNumber; - } - - /** - * @param storeFolderWithLoadNumber The storeFolderWithLoadNumber to set. - */ - public void setStoreFolderWithLoadNumber(String storeFolderWithLoadNumber) { - this.storeFolderWithLoadNumber = storeFolderWithLoadNumber; - } - - /** - * @return Returns the dictionaryCaches. - */ - public Map getDictionaryCaches() { - return dictionaryCaches; - } - - /** - * @param dictionaryCaches The dictionaryCaches to set. - */ - public void setDictionaryCaches(Map dictionaryCaches) { - this.dictionaryCaches = dictionaryCaches; - } - - /** - * @return Returns the timeDimCache. - */ - public Map> getTimeDimCache() { - return timeDimCache; - } - - /** - * @param timeDimCache The timeDimCache to set. - */ - public void setTimeDimCache(Map> timeDimCache) { - this.timeDimCache = timeDimCache; - } - - /** - * @return Returns the dimsFiles. - */ - public String[] getDimsFiles() { - return dimsFiles; - } - - /** - * @param dimsFiles The dimsFiles to set. - */ - public void setDimsFiles(String[] dimsFiles) { - this.dimsFiles = dimsFiles; - } - - /** - * @return Returns the hierCache. - */ - public Map> getHierCache() { - return hierCache; - } - - /** - * @param hierCache The hierCache to set. - */ - public void setHierCache(Map> hierCache) { - this.hierCache = hierCache; - } - - /** - * @return Returns the timDimMax. - */ - public int[] getTimDimMax() { - return timDimMax; - } - - /** - * @param timDimMax The timDimMax to set. - */ - public void setTimDimMax(int[] timDimMax) { - this.timDimMax = timDimMax; - } - - /** - * @return the hierCacheReverse - */ - public Map> getHierCacheReverse() { - return hierCacheReverse; - } - - /** - * @param hierCacheReverse the hierCacheReverse to set - */ - public void setHierCacheReverse(Map> hierCacheReverse) { - this.hierCacheReverse = hierCacheReverse; - } - - public int[] getMax() { - return max; - } - - public void setMax(int[] max) { - this.max = max; - } - - /** - * @return the measureMaxSurroagetMap - */ - public Map getMeasureMaxSurroagetMap() { - return measureMaxSurroagetMap; - } - - /** - * @param measureMaxSurroagetMap the measureMaxSurroagetMap to set - */ - public void setMeasureMaxSurroagetMap(Map measureMaxSurroagetMap) { - this.measureMaxSurroagetMap = measureMaxSurroagetMap; - } - - /** - * @return - */ - public CarbonDimension[] getDimensionOrdinalToDimensionMapping() { - return dimensionOrdinalToDimensionMapping; - } - - /** - * @param dimensionOrdinalToDimensionMapping - */ - public void setDimensionOrdinalToDimensionMapping( - CarbonDimension[] dimensionOrdinalToDimensionMapping) { - this.dimensionOrdinalToDimensionMapping = dimensionOrdinalToDimensionMapping; - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java deleted file mode 100644 index d78dc1a7ec3..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * 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.surrogatekeysgenerator.csvbased; - -import java.util.HashMap; -import java.util.Map; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.keygenerator.KeyGenerator; - -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.trans.step.BaseStepData; -import org.pentaho.di.trans.step.StepDataInterface; - -public class CarbonCSVBasedSeqGenData extends BaseStepData implements StepDataInterface { - - /** - * outputRowMeta - */ - private RowMetaInterface outputRowMeta; - - /** - * surrogateKeyGen - */ - private CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen; - - /** - * keyGenerators - */ - private Map keyGenerators = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - /** - * columnIndex - */ - private Map columnIndex = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - /** - * precomputed default objects - */ - private Object[] defaultObjects; - - /** - * generator - */ - private KeyGenerator generator; - - /** - * the size of the input rows - */ - private int inputSize; - - - public CarbonCSVBasedSeqGenData() { - super(); - } - - /** - * @return Returns the surrogateKeyGen. - */ - public CarbonCSVBasedDimSurrogateKeyGen getSurrogateKeyGen() { - return surrogateKeyGen; - } - - /** - * @param surrogateKeyGen The surrogateKeyGen to set. - */ - public void setSurrogateKeyGen(CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) { - this.surrogateKeyGen = surrogateKeyGen; - } - - /** - * @param inputSize The inputSize to set. - */ - public void setInputSize(int inputSize) { - this.inputSize = inputSize; - } - - /** - * @param generator The generator to set. - */ - public void setGenerator(KeyGenerator generator) { - this.generator = generator; - } - - /** - * @return Returns the keyGenerators. - */ - public Map getKeyGenerators() { - return keyGenerators; - } - - /** - * @return Returns the outputRowMeta. - */ - public RowMetaInterface getOutputRowMeta() { - return outputRowMeta; - } - - /** - * @param outputRowMeta The outputRowMeta to set. - */ - public void setOutputRowMeta(RowMetaInterface outputRowMeta) { - this.outputRowMeta = outputRowMeta; - } - - public void clean() { - outputRowMeta = null; - - surrogateKeyGen = null; - - generator = null; - keyGenerators = null; - - columnIndex = null; - - defaultObjects = null; - - } -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java deleted file mode 100644 index 2d85afa85e7..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java +++ /dev/null @@ -1,1436 +0,0 @@ -/* - * 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.surrogatekeysgenerator.csvbased; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; -import org.apache.carbondata.processing.datatypes.ArrayDataType; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.datatypes.PrimitiveDataType; -import org.apache.carbondata.processing.datatypes.StructDataType; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper; -import org.apache.carbondata.processing.schema.metadata.HierarchiesInfo; -import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; -import org.apache.carbondata.processing.util.NonDictionaryUtil; - -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Counter; -import org.pentaho.di.core.database.DatabaseMeta; -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.exception.KettleValueException; -import org.pentaho.di.core.exception.KettleXMLException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.xml.XMLHandler; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.repository.ObjectId; -import org.pentaho.di.repository.Repository; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStepMeta; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; -import org.w3c.dom.Node; - -public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaInterface { - - /** - * pkg - */ - private static Class pkg = CarbonCSVBasedSeqGenMeta.class; - /** - * Foreign key and respective hierarchy Map - */ - protected Map foreignKeyHierarchyMap; - /** - * hier name - */ - protected String[] hierNames; - /** - * dims - */ - protected int[] dims; - /** - * dims - */ - protected Map complexTypes = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - protected Map> columnProperties = - new HashMap>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - /** - * dimLens - */ - protected int[] dimLens; - /** - * msrs - */ - protected int[] msrs; - /** - * hirches - */ - protected Map hirches; - /** - * Hierarchy Column Name map - */ - protected Map hierColumnMap; - /** - * dateFormat - */ - protected String dateFormat; - /** - * timeDimeIndex - */ - protected int timeDimeIndex = -1; - /** - * timeOrdinalCols - */ - protected String[] timeOrdinalCols = new String[0]; - /** - * timeOrdinalIndices - */ - protected int[] timeOrdinalIndices = new int[0]; - /** - * dimColNames - */ - protected String[] dimColNames; - /** - * - */ - protected String[] noDictionaryCols; - /** - * - */ - protected Map dimColDataTypes; - /** - * measureColumn - */ - protected String[] measureColumn; - /** - * array of carbon measures - */ - protected CarbonMeasure[] carbonMeasures; - /** - * msrMapping - */ - protected int[] msrMapping; - /** - * dims - */ - protected boolean[] dimPresent; - protected int normLength; - /** - * actualDimArrays - */ - protected String[] actualDimArray; - /** - * mrsAggregators - */ - protected String[] msrAggregators; - /** - * columnAndTableName_ColumnMapForAgg - */ - protected Map columnAndTableNameColumnMapForAggMap; - /** - * denormColumnList - */ - protected List denormColumnList; - /** - * Member Mapping - */ - protected int[] memberMapping; - protected Map foreignKeyPrimaryKeyMap; - /** - * carbondim - */ - private String carbondim; - /** - * carbonProps - */ - private String carbonProps; - /** - * carbonmsr - */ - private String carbonmsr; - /** - * carbonhier - */ - private String carbonhier; - /** - * carbonMeasureNames - */ - private String carbonMeasureNames; - /** - * carbonhierColumn - */ - private String carbonhierColumn; - /** - * carbonTime - */ - private String carbonTime; - private String noDictionaryDims; - /** - * carbonSchema - */ - private String carbonSchema; - /** - * batchSize - */ - private int batchSize = 10000; - /** - * isAggregate - */ - private boolean isAggregate; - /** - * generateDimFiles - */ - private boolean generateDimFiles; - /** - * storeType - */ - private String storeType; - /** - * metaHeirSQLQuery - */ - private String metaHeirSQLQuery; - /** - * Foreign key and respective hierarchy String - */ - private String foreignKeyHierarchyString; - /** - * heirNadDimsLensString - */ - private String heirNadDimsLensString; - /** - * measureDataType - */ - private String measureDataType; - /** - * measureSurrogateRequired - */ - private Map measureSurrogateRequired; - private String heirKeySize; - /** - * checkPointFileExits - */ - private String complexDelimiterLevel1; - private String complexDelimiterLevel2; - private String complexTypeString; - - private String columnPropertiesString; - - private String[] complexTypeColumns; - /** - * Primary Key String - */ - private String primaryKeysString; - /** - * foreign key Column name string - */ - private String forgienKeyPrimayKeyString; - /** - * Primary Key Map. - */ - private Map primaryKeyMap; - /** - * - */ - private Map hierDimTableMap; - /** - * propColumns - */ - private List[] propColumns; - /** - * propTypes - */ - private List[] propTypes; - /** - * propIndxs - */ - private int[][] propIndxs; - /** - * metahierVoList - */ - private List metahierVoList; - /** - * dimesionTableNames - */ - private String dimesionTableNames; - /** - * column Ids of dimensions in a table - */ - private String dimensionColumnIds; - /** - * dimTableArray - */ - private String[] dimTableArray; - /** - * tableName - */ - private String tableName; - /** - * MOdified Dimension - */ - private String[] modifiedDimension; - /** - * actualDimNames - */ - private String actualDimNames; - private String normHiers; - /** - * msrAggregatorString - */ - private String msrAggregatorString; - /** - * columnAndTableName_ColumnMapForAggString - */ - private String columnAndTableNameColumnMapForAggString; - private String connectionURL; - private String driverClass; - private String userName; - private String password; - /** - * denormColumNames - */ - private String denormColumNames; - - /** - * databaseName - */ - private String databaseName; - /** - * partitionID - */ - private String partitionID; - - /** - * Id of the load folder - */ - private String segmentId; - - /*** - * String of columns ordinal and column datatype separated by HASH_SPC_CHARACTER - */ - private String columnSchemaDetails; - - /** - * String of key value pair separated by , and HASH_SPC_CHARACTER - */ - private String tableOption; - - /** - * wrapper object having the columnSchemaDetails - */ - private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper; - - /** - * Wrapper object holding the table options - */ - private TableOptionWrapper tableOptionWrapper; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - /** - * column data type string. - */ - private String columnsDataTypeString; - - public CarbonCSVBasedSeqGenMeta() { - super(); - } - - public Map getComplexTypes() { - return complexTypes; - } - - public String getComplexDelimiterLevel1() { - return complexDelimiterLevel1; - } - - public void setComplexDelimiterLevel1(String complexDelimiterLevel1) { - this.complexDelimiterLevel1 = complexDelimiterLevel1; - } - - public String getComplexDelimiterLevel2() { - return complexDelimiterLevel2; - } - - public void setComplexDelimiterLevel2(String complexDelimiterLevel2) { - this.complexDelimiterLevel2 = complexDelimiterLevel2; - } - - public void setComplexTypeString(String complexTypeString) { - this.complexTypeString = complexTypeString; - } - - public void setColumnPropertiesString(String columnPropertiesString) { - this.columnPropertiesString = columnPropertiesString; - } - - public String[] getComplexTypeColumns() { - return complexTypeColumns; - } - - public void setMetaMetaHeirSQLQueries(String metaHeirSQLQuery) { - this.metaHeirSQLQuery = metaHeirSQLQuery; - } - - public boolean isAggregate() { - return isAggregate; - } - - public void setAggregate(boolean isAggregate) { - this.isAggregate = isAggregate; - } - - public int getBatchSize() { - return batchSize; - } - - public void setBatchSize(int batchSize) { - this.batchSize = batchSize; - } - - public String getStoreType() { - return storeType; - } - - public List getMetahierVoList() { - return metahierVoList; - } - - public String getCarbonTime() { - return carbonTime; - } - - // getters and setters for the step settings - - public void setCarbonProps(String carbonProps) { - this.carbonProps = carbonProps; - } - - public void setCarbonmsr(String carbonmsr) { - this.carbonmsr = carbonmsr; - } - - public void setCarbondim(String carbondim) { - this.carbondim = carbondim; - } - - public void setCarbonhier(String carbonhier) { - this.carbonhier = carbonhier; - } - - /** - * @param connectionURL the connectionURL to set - */ - public void setConnectionURL(String connectionURL) { - this.connectionURL = connectionURL; - } - - //TODO SIMIAN - - /** - * @param driverClass the driverClass to set - */ - public void setDriverClass(String driverClass) { - this.driverClass = driverClass; - } - - /** - * @param userName the userName to set - */ - public void setUserName(String userName) { - this.userName = userName; - } - - /** - * @param password the password to set - */ - public void setPassword(String password) { - this.password = password; - } - - public void setDateFormat(String dateFormat) { this.dateFormat = dateFormat; } - - /** - * set sensible defaults for a new step - * - * @see StepMetaInterface#setDefault() - */ - public void setDefault() { - carbonProps = ""; - carbondim = ""; - carbonmsr = ""; - carbonhier = ""; - carbonTime = ""; - driverClass = ""; - connectionURL = ""; - userName = ""; - password = ""; - carbonSchema = ""; - storeType = ""; - isAggregate = false; - metaHeirSQLQuery = ""; - dimesionTableNames = ""; - dimensionColumnIds = ""; - noDictionaryDims = ""; - tableName = ""; - carbonhierColumn = ""; - foreignKeyHierarchyString = ""; - complexTypeString = ""; - columnPropertiesString = ""; - complexDelimiterLevel1 = ""; - complexDelimiterLevel2 = ""; - primaryKeysString = ""; - carbonMeasureNames = ""; - actualDimNames = ""; - normHiers = ""; - msrAggregatorString = ""; - heirKeySize = ""; - heirNadDimsLensString = ""; - measureDataType = ""; - columnAndTableNameColumnMapForAggString = ""; - denormColumNames = ""; - partitionID = ""; - segmentId = ""; - taskNo = ""; - columnSchemaDetails = ""; - columnsDataTypeString = ""; - tableOption = ""; - dateFormat = ""; - } - - // helper method to allocate the arrays - public void allocate(int nrkeys) { - - } - - public String getXML() throws KettleValueException { - StringBuilder retval = new StringBuilder(150); - retval.append(" ").append(XMLHandler.addTagValue("carbonProps", carbonProps)); - retval.append(" ").append(XMLHandler.addTagValue("dim", carbondim)); - retval.append(" ").append(XMLHandler.addTagValue("msr", carbonmsr)); - retval.append(" ").append(XMLHandler.addTagValue("hier", carbonhier)); - retval.append(" ").append(XMLHandler.addTagValue("time", carbonTime)); - retval.append(" ").append(XMLHandler.addTagValue("driverClass", driverClass)); - retval.append(" ").append(XMLHandler.addTagValue("connectionURL", connectionURL)); - retval.append(" ").append(XMLHandler.addTagValue("userName", userName)); - retval.append(" ").append(XMLHandler.addTagValue("password", password)); - retval.append(" ").append(XMLHandler.addTagValue("batchSize", batchSize)); - retval.append(" ").append(XMLHandler.addTagValue("genDimFiles", generateDimFiles)); - retval.append(" ").append(XMLHandler.addTagValue("isAggregate", isAggregate)); - retval.append(" ").append(XMLHandler.addTagValue("storeType", storeType)); - retval.append(" ").append(XMLHandler.addTagValue("metadataFilePath", metaHeirSQLQuery)); - retval.append(" ") - .append(XMLHandler.addTagValue("foreignKeyHierarchyString", foreignKeyHierarchyString)); - retval.append(" ").append(XMLHandler.addTagValue("complexTypeString", complexTypeString)); - retval.append(" ") - .append(XMLHandler.addTagValue("columnPropertiesString", columnPropertiesString)); - retval.append(" ") - .append(XMLHandler.addTagValue("complexDelimiterLevel1", complexDelimiterLevel1)); - retval.append(" ") - .append(XMLHandler.addTagValue("complexDelimiterLevel2", complexDelimiterLevel2)); - retval.append(" ").append(XMLHandler.addTagValue("dateFormat", dateFormat)); - retval.append(" ").append(XMLHandler.addTagValue("primaryKeysString", primaryKeysString)); - retval.append(" ").append(XMLHandler.addTagValue("carbonMeasureNames", carbonMeasureNames)); - retval.append(" ").append(XMLHandler.addTagValue("actualDimNames", actualDimNames)); - retval.append(" ") - .append(XMLHandler.addTagValue("msrAggregatorString", msrAggregatorString)); - - retval.append(" ").append(XMLHandler.addTagValue("dimHierReleation", dimesionTableNames)); - retval.append(" ").append(XMLHandler.addTagValue("dimensionColumnIds", dimensionColumnIds)); - retval.append(" ").append(XMLHandler.addTagValue("dimNoDictionary", noDictionaryDims)); - retval.append(" ").append(XMLHandler.addTagValue("dimColDataTypes", columnsDataTypeString)); - retval.append(" ").append(XMLHandler.addTagValue("factOrAggTable", tableName)); - retval.append(" ").append(XMLHandler.addTagValue("carbonhierColumn", carbonhierColumn)); - retval.append(" ").append(XMLHandler.addTagValue("normHiers", normHiers)); - retval.append(" ").append(XMLHandler.addTagValue("heirKeySize", heirKeySize)); - - retval.append(" ") - .append(XMLHandler.addTagValue("forgienKeyPrimayKeyString", forgienKeyPrimayKeyString)); - retval.append(" ") - .append(XMLHandler.addTagValue("heirNadDimsLensString", heirNadDimsLensString)); - retval.append(" ").append(XMLHandler.addTagValue("measureDataType", measureDataType)); - retval.append(" ").append(XMLHandler.addTagValue("columnAndTableName_ColumnMapForAggString", - columnAndTableNameColumnMapForAggString)); - retval.append(" ").append(XMLHandler.addTagValue("databaseName", databaseName)); - retval.append(" ").append(XMLHandler.addTagValue("tableName", tableName)); - retval.append(" ").append(XMLHandler.addTagValue("denormColumNames", denormColumNames)); - retval.append(" ").append(XMLHandler.addTagValue("partitionID", partitionID)); - retval.append(" ").append(XMLHandler.addTagValue("segmentId", segmentId)); - retval.append(" ").append(XMLHandler.addTagValue("taskNo", taskNo)); - retval.append(" ") - .append(XMLHandler.addTagValue("columnSchemaDetails", columnSchemaDetails)); - retval.append(" ") - .append(XMLHandler.addTagValue("tableOption", tableOption)); - return retval.toString(); - } - - public void loadXML(Node stepnode, List databases, Map counters) - throws KettleXMLException { - - try { - - carbonProps = XMLHandler.getTagValue(stepnode, "carbonProps"); - carbondim = XMLHandler.getTagValue(stepnode, "dim"); - carbonmsr = XMLHandler.getTagValue(stepnode, "msr"); - carbonhier = XMLHandler.getTagValue(stepnode, "hier"); - carbonTime = XMLHandler.getTagValue(stepnode, "time"); - driverClass = XMLHandler.getTagValue(stepnode, "driverClass"); - connectionURL = XMLHandler.getTagValue(stepnode, "connectionURL"); - userName = XMLHandler.getTagValue(stepnode, "userName"); - password = XMLHandler.getTagValue(stepnode, "password"); - carbonhierColumn = XMLHandler.getTagValue(stepnode, "carbonhierColumn"); - foreignKeyHierarchyString = XMLHandler.getTagValue(stepnode, "foreignKeyHierarchyString"); - complexTypeString = XMLHandler.getTagValue(stepnode, "complexTypeString"); - columnPropertiesString = XMLHandler.getTagValue(stepnode, "columnPropertiesString"); - complexDelimiterLevel1 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel1"); - complexDelimiterLevel2 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel2"); - dateFormat = XMLHandler.getTagValue(stepnode, "dateFormat"); - primaryKeysString = XMLHandler.getTagValue(stepnode, "primaryKeysString"); - carbonMeasureNames = XMLHandler.getTagValue(stepnode, "carbonMeasureNames"); - actualDimNames = XMLHandler.getTagValue(stepnode, "actualDimNames"); - normHiers = XMLHandler.getTagValue(stepnode, "normHiers"); - msrAggregatorString = XMLHandler.getTagValue(stepnode, "msrAggregatorString"); - heirKeySize = XMLHandler.getTagValue(stepnode, "heirKeySize"); - forgienKeyPrimayKeyString = XMLHandler.getTagValue(stepnode, "forgienKeyPrimayKeyString"); - heirNadDimsLensString = XMLHandler.getTagValue(stepnode, "heirNadDimsLensString"); - measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType"); - columnAndTableNameColumnMapForAggString = - XMLHandler.getTagValue(stepnode, "columnAndTableName_ColumnMapForAggString"); - dimesionTableNames = XMLHandler.getTagValue(stepnode, "dimHierReleation"); - dimensionColumnIds = XMLHandler.getTagValue(stepnode, "dimensionColumnIds"); - noDictionaryDims = XMLHandler.getTagValue(stepnode, "dimNoDictionary"); - columnsDataTypeString = XMLHandler.getTagValue(stepnode, "dimColDataTypes"); - tableName = XMLHandler.getTagValue(stepnode, "factOrAggTable"); - tableName = XMLHandler.getTagValue(stepnode, "tableName"); - databaseName = XMLHandler.getTagValue(stepnode, "databaseName"); - denormColumNames = XMLHandler.getTagValue(stepnode, "denormColumNames"); - partitionID = XMLHandler.getTagValue(stepnode, "partitionID"); - segmentId = XMLHandler.getTagValue(stepnode, "segmentId"); - taskNo = XMLHandler.getTagValue(stepnode, "taskNo"); - columnSchemaDetails = XMLHandler.getTagValue(stepnode, "columnSchemaDetails"); - tableOption = XMLHandler.getTagValue(stepnode, "tableOption"); - String batchConfig = XMLHandler.getTagValue(stepnode, "batchSize"); - - if (batchConfig != null) { - batchSize = Integer.parseInt(batchConfig); - } - - String dimeFileConfig = XMLHandler.getTagValue(stepnode, "genDimFiles"); - if (dimeFileConfig != null) { - generateDimFiles = Boolean.parseBoolean(dimeFileConfig); - } - - storeType = XMLHandler.getTagValue(stepnode, "storeType"); - metaHeirSQLQuery = XMLHandler.getTagValue(stepnode, "metadataFilePath"); - - isAggregate = "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, "isAggregate")); - - int nrKeys = XMLHandler.countNodes(stepnode, "lookup"); - allocate(nrKeys); - - } catch (Exception e) { - throw new KettleXMLException("Template Plugin Unable to read step info from XML node", e); - } - - } - - public void initialize() throws KettleException { - this.columnSchemaDetailsWrapper = new ColumnSchemaDetailsWrapper(columnSchemaDetails); - this.tableOptionWrapper = TableOptionWrapper.getTableOptionWrapperInstance(); - tableOptionWrapper.populateTableOptions(tableOption); - - updateDimensions(carbondim, carbonmsr, noDictionaryDims); - dimColDataTypes = NonDictionaryUtil.extractDimColsDataTypeValues(columnsDataTypeString); - if (null != complexTypeString) { - complexTypes = getComplexTypesMap(complexTypeString); - } else { - complexTypeColumns = new String[0]; - } - - if (null != columnPropertiesString) { - updateColumnPropertiesMap(columnPropertiesString); - } - hirches = getHierarichies(carbonhier); - - hierColumnMap = getHierarchiesColumnMap(carbonhierColumn); - - foreignKeyHierarchyMap = getForeignKeyHierMap(foreignKeyHierarchyString); - - primaryKeyMap = updatePrimaryKeyMap(primaryKeysString); - - foreignKeyPrimaryKeyMap = getForeignKeyColumnNameMap(forgienKeyPrimayKeyString); - - actualDimArray = getActualDimensionArray(actualDimNames); - - //update non time dimension properties - updateDimProperties(); - - updateMeasureAggregator(msrAggregatorString); - - measureSurrogateRequired = getMeasureDatatypeMap(measureDataType); - - updateHierDimTableMap(dimesionTableNames); - - if (isAggregate) { - columnAndTableNameColumnMapForAggMap = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - String[] split = - columnAndTableNameColumnMapForAggString.split(CarbonCommonConstants.HASH_SPC_CHARACTER); - for (int i = 0; i < split.length; i++) { - String[] split2 = split[i].split(CarbonCommonConstants.HYPHEN_SPC_CHARACTER); - columnAndTableNameColumnMapForAggMap.put(split2[0], split2[1]); - } - } - - updateDenormColunList(denormColumNames); - } - - private void updateColumnPropertiesMap(String columnPropertiesString) { - String[] colsProperty = columnPropertiesString.split(CarbonCommonConstants.HASH_SPC_CHARACTER); - for (String property : colsProperty) { - String[] colKeyVals = property.split(CarbonCommonConstants.COLON_SPC_CHARACTER); - String colName = colKeyVals[0]; - Map colPropMap = new HashMap<>(); - String[] keyVals = colKeyVals[1].split(CarbonCommonConstants.COMA_SPC_CHARACTER); - for (int i = 0; i < keyVals.length; i++) { - String[] keyVal = keyVals[i].split(CarbonCommonConstants.HYPHEN_SPC_CHARACTER); - String key = keyVal[0]; - String value = keyVal[1]; - colPropMap.put(key, value); - } - columnProperties.put(colName, colPropMap); - } - } - - private void updateDenormColunList(String denormColumNames) { - // - if (null == denormColumNames || "".equals(denormColumNames)) { - denormColumnList = new ArrayList(1); - return; - } - - String[] columnNames = denormColumNames.split(CarbonCommonConstants.HASH_SPC_CHARACTER); - - if (null == denormColumnList) { - denormColumnList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - } - - for (String colName : columnNames) { - denormColumnList.add(colName); - } - } - - private void updateHierDimTableMap(String dimesionTableNames) { - if (null == dimesionTableNames || "".equals(dimesionTableNames)) { - return; - } - - String[] hierTableName = - dimesionTableNames.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - if (null == hierDimTableMap) { - hierDimTableMap = new HashMap(hierTableName.length); - } - if (null == dimTableArray) { - dimTableArray = new String[hierTableName.length]; - } - int i = 0; - for (String hierTable : hierTableName) { - String[] hierAndTable = hierTable.split(CarbonCommonConstants.COLON_SPC_CHARACTER); - hierDimTableMap.put(hierAndTable[0], hierAndTable[1]); - dimTableArray[i++] = hierAndTable[1]; - } - } - - private Map getMeasureDatatypeMap(String measureDataType) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - - private void updateMeasureAggregator(String msrAggregatorString) { - String[] split = null; - if (msrAggregatorString == null) { - split = new String[0]; - } else { - split = msrAggregatorString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER); - } - msrAggregators = new String[split.length]; - System.arraycopy(split, 0, msrAggregators, 0, split.length); - } - - private String[] getActualDimensionArray(String actualDimNames) { - if (actualDimNames == null || "".equals(actualDimNames)) { - return new String[0]; - } - - return actualDimNames.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - } - - private List getNormHierList(String normHier) { - List hierList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - if (null != normHier && normHier.length() != 0) { - String[] split = normHier.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - - for (int i = 0; i < split.length; i++) { - hierList.add(split[i]); - } - } - return hierList; - } - - private Map getForeignKeyColumnNameMap(String foreignKeyColumnNameString) { - if (foreignKeyColumnNameString == null || "".equals(foreignKeyColumnNameString)) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - Map map = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - String[] foreignKeys = - foreignKeyColumnNameString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < foreignKeys.length; i++) { - String[] foreignHierArray = foreignKeys[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - String hiers = map.get(foreignHierArray[0]); - - if (null == hiers) { - map.put(foreignHierArray[0], foreignHierArray[1]); - } else { - map.put(foreignHierArray[0], - hiers + CarbonCommonConstants.COMA_SPC_CHARACTER + foreignHierArray[1]); - } - } - return map; - } - - private Map updatePrimaryKeyMap(String primaryKeysString) { - if (primaryKeysString == null || "".equals(primaryKeysString)) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - Map resultMap = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - String[] primaryKeys = primaryKeysString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < primaryKeys.length; i++) { - resultMap.put(primaryKeys[i], true); - } - return resultMap; - } - - public void updateHierMappings(RowMetaInterface metaInterface) { - List actualHierList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - for (int j = 0; j < metaInterface.size(); j++) { - String foreignKey = metaInterface.getValueMeta(j).getName(); - String actualHier = foreignKeyHierarchyMap.get(foreignKey); - if (null != actualHier) { - if (actualHier.contains(CarbonCommonConstants.COMA_SPC_CHARACTER)) { - String[] splitHier = actualHier.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - for (String hier : splitHier) { - actualHierList.add(hier); - } - } else { - actualHierList.add(actualHier); - } - } - } - - hierNames = new String[actualHierList.size()]; - hierNames = actualHierList.toArray(new String[actualHierList.size()]); - } - - private Map getComplexTypesMap(String complexTypeString) { - Map complexTypesMap = new LinkedHashMap(); - String[] hierarchies = complexTypeString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER); - complexTypeColumns = new String[hierarchies.length]; - for (int i = 0; i < hierarchies.length; i++) { - String[] levels = hierarchies[i].split(CarbonCommonConstants.HASH_SPC_CHARACTER); - String[] levelInfo = levels[0].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - GenericDataType g = levelInfo[1].equals(CarbonCommonConstants.ARRAY) ? - new ArrayDataType(levelInfo[0], "", levelInfo[3]) : - new StructDataType(levelInfo[0], "", levelInfo[3]); - complexTypesMap.put(levelInfo[0], g); - complexTypeColumns[i] = levelInfo[0]; - for (int j = 1; j < levels.length; j++) { - levelInfo = levels[j].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - switch (levelInfo[1]) { - case CarbonCommonConstants.ARRAY: - g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3])); - break; - case CarbonCommonConstants.STRUCT: - g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3])); - break; - default: - g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3], - Integer.parseInt(levelInfo[4]))); - } - } - } - return complexTypesMap; - } - - private Map getForeignKeyHierMap(String foreignKeyHierarchyString) { - if (foreignKeyHierarchyString == null || "".equals(foreignKeyHierarchyString)) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - Map map = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - String[] hies = foreignKeyHierarchyString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < hies.length; i++) { - String[] foreignHierArray = hies[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - String hiers = map.get(foreignHierArray[0]); - - if (null == hiers) { - map.put(foreignHierArray[0], foreignHierArray[1]); - } else { - map.put(foreignHierArray[0], - hiers + CarbonCommonConstants.COMA_SPC_CHARACTER + foreignHierArray[1]); - } - - } - return map; - } - - private Map getHierarchiesColumnMap(String carbonhierColumn) { - if (carbonhierColumn == null || "".equals(carbonhierColumn)) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - Map map = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - String[] hies = carbonhierColumn.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < hies.length; i++) { - String hie = hies[i]; - - String hierName = hie.substring(0, hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER)); - - String[] columnArray = getStringArray(hie.substring( - hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER) - + CarbonCommonConstants.COLON_SPC_CHARACTER.length(), hie.length())); - map.put(hierName, columnArray); - } - return map; - } - - private String[] getStringArray(String columnNames) { - String[] splitedColumnNames = columnNames.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - String[] columns = new String[splitedColumnNames.length]; - - System.arraycopy(splitedColumnNames, 0, columns, 0, columns.length); - return columns; - } - - /** - * Parse the properties string. - * Level Entries separated by '&' - * Level and prop details separated by ':' - * Property column name and index separated by ',' - * Level:p1,index1:p2,index2&Level2.... - */ - private void updateDimProperties() { - Map indices = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - Map columns = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - Map dbTypes = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - if (carbonProps != null && !"".equals(carbonProps)) { - String[] entries = carbonProps.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - for (int i = 0; i < entries.length; i++) { - String[] levelEntry = entries[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - String dimColumnName = levelEntry[0]; - int[] pIndices = new int[levelEntry.length - 1]; - String[] cols = new String[levelEntry.length - 1]; - String[] dbType = new String[levelEntry.length - 1]; - for (int j = 1; j < levelEntry.length; j++) { - String[] propEntry = levelEntry[j].split(CarbonCommonConstants.COMA_SPC_CHARACTER); - pIndices[j - 1] = Integer.parseInt(propEntry[1]); - - cols[j - 1] = propEntry[0]; - dbType[j - 1] = propEntry[2]; - } - - indices.put(dimColumnName, pIndices); - columns.put(dimColumnName, cols); - dbTypes.put(dimColumnName, dbType); - } - } - - if (indices.isEmpty()) { - return; - } - - propColumns = new List[dimColNames.length]; - propTypes = new List[dimColNames.length]; - propIndxs = new int[dimColNames.length][]; - - //Fill the property details based on the map created - for (int i = 0; i < dimColNames.length; i++) { - //Properties present or not - if (indices.containsKey(dimColNames[i])) { - propColumns[i] = Arrays.asList(columns.get(dimColNames[i])); - propTypes[i] = Arrays.asList(dbTypes.get(dimColNames[i])); - propIndxs[i] = indices.get(dimColNames[i]); - } else { - propColumns[i] = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - propTypes[i] = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - propIndxs[i] = new int[0]; - } - } - } - - private Map getHierarichies(String ds) { - if (ds == null || "".equals(ds)) { - return new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - Map map = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - String[] hies = ds.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - for (int i = 0; i < hies.length; i++) { - String hie = hies[i]; - - String name = hie.substring(0, hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER)); - - int[] a = getIntArray(hie.substring(hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER) - + CarbonCommonConstants.COLON_SPC_CHARACTER.length(), hie.length())); - map.put(name, a); - } - return map; - } - - private int[] getIntArray(String ds) { - - String[] sp = ds.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - int[] a = new int[sp.length]; - - for (int i = 0; i < a.length; i++) { - a[i] = Integer.parseInt(sp[i]); - } - return a; - - } - - private void updateDimensions(String ds, String msr, String noDictionaryDims) { - String[] sp = null; - if (null != ds) { - sp = ds.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - } else { - sp = new String[0]; - } - int[] dimsLocal = new int[sp.length]; - int[] lens = new int[sp.length]; - List list = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - dimPresent = new boolean[sp.length]; - - for (int i = 0; i < dimsLocal.length; i++) { - String[] dim = sp[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - list.add(dim[0]); - dimsLocal[i] = Integer.parseInt(dim[1]); - lens[i] = Integer.parseInt(dim[2]); - - if ("Y".equals(dim[3])) { - dimPresent[i] = true; - normLength++; - } - } - dims = dimsLocal; - dimLens = lens; - dimColNames = list.toArray(new String[list.size()]); - - // get high cardinality dimension Array - noDictionaryCols = NonDictionaryUtil.extractNoDictionaryDimsArr(noDictionaryDims); - - String[] sm = null; - if (null != msr) { - sm = msr.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - } else { - sm = new String[0]; - } - int[] m = new int[sm.length]; - Set mlist = new LinkedHashSet(); - for (int i = 0; i < m.length; i++) { - String[] ms = sm[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER); - mlist.add(ms[0]); - m[i] = Integer.parseInt(ms[1]); - } - msrs = m; - measureColumn = mlist.toArray(new String[mlist.size()]); - } - - public void readRep(Repository rep, ObjectId idStep, List databases, - Map counters) throws KettleException { - try { - // - carbonProps = rep.getStepAttributeString(idStep, "carbonProps"); - carbonmsr = rep.getStepAttributeString(idStep, "msr"); - carbondim = rep.getStepAttributeString(idStep, "dim"); - carbonhier = rep.getStepAttributeString(idStep, "hier"); - carbonTime = rep.getStepAttributeString(idStep, "time"); - // - driverClass = rep.getStepAttributeString(idStep, "driverClass"); - connectionURL = rep.getStepAttributeString(idStep, "connectionURL"); - userName = rep.getStepAttributeString(idStep, "userName"); - password = rep.getStepAttributeString(idStep, "password"); - isAggregate = rep.getStepAttributeBoolean(idStep, "isAggregate"); - metaHeirSQLQuery = rep.getStepAttributeString(idStep, "metadataFilePath"); - carbonhierColumn = rep.getStepAttributeString(idStep, "carbonhierColumn"); - foreignKeyHierarchyString = rep.getStepAttributeString(idStep, "foreignKeyHierarchyString"); - primaryKeysString = rep.getStepAttributeString(idStep, "primaryKeysString"); - carbonMeasureNames = rep.getStepAttributeString(idStep, "carbonMeasureNames"); - actualDimNames = rep.getStepAttributeString(idStep, "actualDimNames"); - msrAggregatorString = rep.getStepAttributeString(idStep, "msrAggregatorString"); - - dimesionTableNames = rep.getStepAttributeString(idStep, "dimHierReleation"); - dimensionColumnIds = rep.getStepAttributeString(idStep, "dimensionColumnIds"); - noDictionaryDims = rep.getStepAttributeString(idStep, "dimNoDictionary"); - columnsDataTypeString = rep.getStepAttributeString(idStep, "dimColDataTypes"); - normHiers = rep.getStepAttributeString(idStep, "normHiers"); - tableName = rep.getStepAttributeString(idStep, "factOrAggTable"); - batchSize = Integer.parseInt(rep.getStepAttributeString(idStep, "batchSize")); - heirKeySize = rep.getStepAttributeString(idStep, "heirKeySize"); - forgienKeyPrimayKeyString = rep.getStepAttributeString(idStep, "forgienKeyPrimayKeyString"); - heirNadDimsLensString = rep.getStepAttributeString(idStep, "heirNadDimsLensString"); - measureDataType = rep.getStepAttributeString(idStep, "measureDataType"); - columnAndTableNameColumnMapForAggString = - rep.getStepAttributeString(idStep, "columnAndTableName_ColumnMapForAggString"); - databaseName = rep.getStepAttributeString(idStep, "databaseName"); - - tableName = rep.getStepAttributeString(idStep, "tableName"); - denormColumNames = rep.getStepAttributeString(idStep, "denormColumNames"); - partitionID = rep.getStepAttributeString(idStep, "partitionID"); - segmentId = rep.getStepAttributeString(idStep, "segmentId"); - taskNo = rep.getStepAttributeString(idStep, "taskNo"); - columnSchemaDetails = rep.getStepAttributeString(idStep, "columnSchemaDetails"); - tableOption = rep.getStepAttributeString(idStep, "tableOption"); - dateFormat = rep.getStepAttributeString(idStep, "dateFormat"); - int nrKeys = rep.countNrStepAttributes(idStep, "lookup_keyfield"); - allocate(nrKeys); - } catch (Exception e) { - throw new KettleException( - BaseMessages.getString(pkg, "CarbonStep.Exception.UnexpectedErrorInReadingStepInfo"), e); - } - } - - public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep) - throws KettleException { - try { - rep.saveStepAttribute(idTransformation, idStep, "dim", carbondim); - rep.saveStepAttribute(idTransformation, idStep, "carbonProps", carbonProps); - rep.saveStepAttribute(idTransformation, idStep, "msr", carbonmsr); - rep.saveStepAttribute(idTransformation, idStep, "hier", carbonhier); - rep.saveStepAttribute(idTransformation, idStep, "carbonhierColumn", carbonhierColumn); - rep.saveStepAttribute(idTransformation, idStep, "columnAndTableName_ColumnMapForAggString", - columnAndTableNameColumnMapForAggString); - rep.saveStepAttribute(idTransformation, idStep, "time", carbonTime); - rep.saveStepAttribute(idTransformation, idStep, "driverClass", driverClass); - rep.saveStepAttribute(idTransformation, idStep, "connectionURL", connectionURL); - rep.saveStepAttribute(idTransformation, idStep, "userName", userName); - rep.saveStepAttribute(idTransformation, idStep, "password", password); - rep.saveStepAttribute(idTransformation, idStep, "isInitialLoad", isAggregate); - rep.saveStepAttribute(idTransformation, idStep, "metadataFilePath", metaHeirSQLQuery); - rep.saveStepAttribute(idTransformation, idStep, "batchSize", batchSize); - rep.saveStepAttribute(idTransformation, idStep, "dimHierReleation", dimesionTableNames); - rep.saveStepAttribute(idTransformation, idStep, "dimensionColumnIds", dimensionColumnIds); - rep.saveStepAttribute(idTransformation, idStep, "dimNoDictionary", noDictionaryDims); - rep.saveStepAttribute(idTransformation, idStep, "dimColDataTypes", columnsDataTypeString); - rep.saveStepAttribute(idTransformation, idStep, "foreignKeyHierarchyString", - foreignKeyHierarchyString); - rep.saveStepAttribute(idTransformation, idStep, "primaryKeysString", primaryKeysString); - rep.saveStepAttribute(idTransformation, idStep, "carbonMeasureNames", carbonMeasureNames); - rep.saveStepAttribute(idTransformation, idStep, "actualDimNames", actualDimNames); - rep.saveStepAttribute(idTransformation, idStep, "normHiers", normHiers); - rep.saveStepAttribute(idTransformation, idStep, "msrAggregatorString", msrAggregatorString); - rep.saveStepAttribute(idTransformation, idStep, "heirKeySize", heirKeySize); - rep.saveStepAttribute(idTransformation, idStep, "forgienKeyPrimayKeyString", - forgienKeyPrimayKeyString); - rep.saveStepAttribute(idTransformation, idStep, "factOrAggTable", tableName); - rep.saveStepAttribute(idTransformation, idStep, "heirNadDimsLensString", - heirNadDimsLensString); - rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType); - rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName); - rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName); - rep.saveStepAttribute(idTransformation, idStep, "denormColumNames", denormColumNames); - rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID); - rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId); - rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo); - rep.saveStepAttribute(idTransformation, idStep, "columnSchemaDetails", columnSchemaDetails); - rep.saveStepAttribute(idTransformation, idStep, "tableOption", tableOption); - rep.saveStepAttribute(idTransformation, idStep, "dateFormat", dateFormat); - } catch (Exception e) { - throw new KettleException( - BaseMessages.getString(pkg, "CarbonStep.Exception.UnableToSaveStepInfoToRepository") - + idStep, e); - } - } - - public void check(List remarks, TransMeta transmeta, StepMeta stepMeta, - RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) { - CarbonDataProcessorUtil.check(pkg, remarks, stepMeta, prev, input); - } - - public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int cnr, - TransMeta transMeta, Trans disp) { - return new CarbonCSVBasedSeqGenStep(stepMeta, stepDataInterface, cnr, transMeta, disp); - } - - public StepDataInterface getStepData() { - return new CarbonCSVBasedSeqGenData(); - } - - public List[] getPropertiesColumns() { - return propColumns; - } - - public int[][] getPropertiesIndices() { - return propIndxs; - } - - public List[] getPropTypes() { - return propTypes; - } - - public void setTableNames(String dimHierReleation) { - this.dimesionTableNames = dimHierReleation; - } - - /** - * @return column Ids - */ - public String[] getDimensionColumnIds() { - return null != dimensionColumnIds ? - dimensionColumnIds.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER) : - new String[0]; - } - - /** - * @param dimensionColumnIds column Ids for dimensions in a table - */ - public void setDimensionColumnIds(String dimensionColumnIds) { - this.dimensionColumnIds = dimensionColumnIds; - } - - public String getTableName() { - return tableName; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void setModifiedDimension(String[] modifiedDimension) { - this.modifiedDimension = modifiedDimension; - } - - public void setCarbonhierColumn(String carbonhierColumn) { - this.carbonhierColumn = carbonhierColumn; - } - - public void setForeignKeyHierarchyString(String foreignKeyHierarchyString) { - this.foreignKeyHierarchyString = foreignKeyHierarchyString; - } - - public void setPrimaryKeysString(String primaryKeysString) { - this.primaryKeysString = primaryKeysString; - } - - public Map getPrimaryKeyMap() { - return primaryKeyMap; - } - - public void setCarbonMeasureNames(String carbonMeasureNames) { - this.carbonMeasureNames = carbonMeasureNames; - } - - public void setActualDimNames(String actualDimNames) { - this.actualDimNames = actualDimNames; - } - - public void setNormHiers(String normHiers) { - this.normHiers = normHiers; - } - - public void setMsrAggregatorString(String msrAggregatorString) { - this.msrAggregatorString = msrAggregatorString; - } - - public void setHeirKeySize(String heirKeySize) { - this.heirKeySize = heirKeySize; - } - - public void setForgienKeyPrimayKeyString(String forgienKeyPrimayKeyString) { - this.forgienKeyPrimayKeyString = forgienKeyPrimayKeyString; - } - - public void setHeirNadDimsLensString(String heirNadDimsLensString) { - this.heirNadDimsLensString = heirNadDimsLensString; - } - - public void setMeasureDataType(String measureDataType) { - this.measureDataType = measureDataType; - } - - public Map getMeasureSurrogateRequired() { - return measureSurrogateRequired; - } - - public Map getHierDimTableMap() { - return hierDimTableMap; - } - - public String[] getDimTableArray() { - return dimTableArray; - } - - public void setColumnAndTableNameColumnMapForAggString( - String columnAndTableNameColumnMapForAggString) { - this.columnAndTableNameColumnMapForAggString = columnAndTableNameColumnMapForAggString; - } - - public String getDatabaseName() { - return databaseName; - } - - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - public void setDenormColumNames(String denormColumNames) { - this.denormColumNames = denormColumNames; - } - - public String getNoDictionaryDims() { - return noDictionaryDims; - } - - public void setNoDictionaryDims(String noDictionaryDims) { - this.noDictionaryDims = noDictionaryDims; - } - - /** - * @param columnsDataTypeString - */ - public void setDimensionColumnsDataType(String columnsDataTypeString) { - this.columnsDataTypeString = columnsDataTypeString; - - } - - /** - * @return partitionId - */ - public String getPartitionID() { - return partitionID; - } - - /** - * @param partitionID - */ - public void setPartitionID(String partitionID) { - this.partitionID = partitionID; - } - - /** - * set the the serialized String of columnSchemaDetails - * - * @param columnSchemaDetails - */ - public void setColumnSchemaDetails(String columnSchemaDetails) { - this.columnSchemaDetails = columnSchemaDetails; - } - - /** - * return segmentId - * - * @return - */ - public String getSegmentId() { - return segmentId; - } - - /** - * set segment Id - * - * @param segmentId - */ - public void setSegmentId(String segmentId) { - this.segmentId = segmentId; - } - - /** - * @param taskNo - */ - public void setTaskNo(String taskNo) { - this.taskNo = taskNo; - } - - /** - * @return - */ - public String getTaskNo() { - return taskNo; - } - - public Map> getColumnPropertiesMap() { - return columnProperties; - } - - /** - * returns wrapper object having the columnSchemaDetails - * - * @return - */ - public ColumnSchemaDetailsWrapper getColumnSchemaDetailsWrapper() { - return columnSchemaDetailsWrapper; - } - - /** - * the method set the TableOption details - * @param tableOption - */ - public void setTableOption(String tableOption) { - this.tableOption = tableOption; - } - - /** - * the method returns the wrapper object of tableoption - * @return - */ - public TableOptionWrapper getTableOptionWrapper() { - return tableOptionWrapper; - } -} - diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java deleted file mode 100644 index f037cf068ad..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java +++ /dev/null @@ -1,2009 +0,0 @@ -/* - * 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.surrogatekeysgenerator.csvbased; - -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.FileChannel; -import java.nio.charset.Charset; -import java.sql.Connection; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.common.logging.impl.StandardLogService; -import org.apache.carbondata.core.cache.dictionary.Dictionary; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator; -import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory; -import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory; -import org.apache.carbondata.core.metadata.CarbonMetadata; -import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.encoder.Encoding; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory; -import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.DataTypeUtil; -import org.apache.carbondata.core.writer.ByteArrayHolder; -import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV; -import org.apache.carbondata.processing.constants.LoggerAction; -import org.apache.carbondata.processing.dataprocessor.manager.CarbonDataProcessorManager; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.mdkeygen.file.FileData; -import org.apache.carbondata.processing.mdkeygen.file.FileManager; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper; -import org.apache.carbondata.processing.schema.metadata.ColumnsInfo; -import org.apache.carbondata.processing.schema.metadata.HierarchiesInfo; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; -import org.apache.carbondata.processing.util.NonDictionaryUtil; -import static org.apache.carbondata.processing.constants.TableOptionConstant.BAD_RECORDS_ACTION; -import static org.apache.carbondata.processing.constants.TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE; -import static org.apache.carbondata.processing.constants.TableOptionConstant.SERIALIZATION_NULL_FORMAT; - -import org.pentaho.di.core.exception.KettleException; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.core.row.ValueMeta; -import org.pentaho.di.core.row.ValueMetaInterface; -import org.pentaho.di.trans.Trans; -import org.pentaho.di.trans.TransMeta; -import org.pentaho.di.trans.step.BaseStep; -import org.pentaho.di.trans.step.StepDataInterface; -import org.pentaho.di.trans.step.StepMeta; -import org.pentaho.di.trans.step.StepMetaInterface; - -public class CarbonCSVBasedSeqGenStep extends BaseStep { - - /** - * BYTE ENCODING - */ - public static final String BYTE_ENCODING = "ISO-8859-1"; - /** - * LOGGER - */ - private static final LogService LOGGER = - LogServiceFactory.getLogService(CarbonCSVBasedSeqGenStep.class.getName()); - /** - * NUM_CORES_DEFAULT_VAL - */ - private static final int NUM_CORES_DEFAULT_VAL = 2; - /** - * drivers - */ - private static final Map DRIVERS; - - static { - - DRIVERS = new HashMap(16); - DRIVERS.put("oracle.jdbc.OracleDriver", CarbonCommonConstants.TYPE_ORACLE); - DRIVERS.put("com.mysql.jdbc.Driver", CarbonCommonConstants.TYPE_MYSQL); - DRIVERS.put("org.gjt.mm.mysql.Driver", CarbonCommonConstants.TYPE_MYSQL); - DRIVERS.put("com.microsoft.sqlserver.jdbc.SQLServerDriver", CarbonCommonConstants.TYPE_MSSQL); - DRIVERS.put("com.sybase.jdbc3.jdbc.SybDriver", CarbonCommonConstants.TYPE_SYBASE); - } - - /** - * ReentrantLock getRowLock - */ - private final Object getRowLock = new Object(); - /** - * ReentrantLock putRowLock - */ - private final Object putRowLock = new Object(); - /** - * CarbonSeqGenData - */ - private CarbonCSVBasedSeqGenData data; - /** - * CarbonSeqGenStepMeta1 - */ - private CarbonCSVBasedSeqGenMeta meta; - /** - * Map of Connection - */ - private Map cons = new HashMap<>(16); - /** - * Csv file path - */ - private String csvFilepath; - - /** - * badRecordsLogger - */ - private BadRecordsLogger badRecordsLogger; - /** - * Normalized Hier and HierWriter map - */ - private Map nrmlizedHierWriterMap = - new HashMap(16); - /** - * load Folder location - */ - private String loadFolderLoc; - /** - * File manager - */ - private IFileManagerComposite filemanager; - /** - * measureCol - */ - private List measureCol; - /** - * dimPresentCsvOrder - Dim present In CSV order - */ - private boolean[] dimPresentCsvOrder; - /** - * propMap - */ - private Map propMap; - /** - * resultArray - */ - private Future[] resultArray; - - /** - * denormHierarchies - */ - private List denormHierarchies; - /** - * readCounter - */ - private long readCounter; - /** - * writeCounter - */ - private long writeCounter; - /** - * logCounter - */ - private int logCounter; - /** - * presentColumnMapIndex - */ - private int[] presentColumnMapIndex; - /** - * measurePresentMapping - */ - private boolean[] measurePresentMapping; - /** - * measureSurrogateReqMapping - */ - private boolean[] measureSurrogateReqMapping; - /** - * foreignKeyMappingColumns - */ - private String[] foreignKeyMappingColumns; - /** - * foreignKeyMappingColumns - */ - private String[][] foreignKeyMappingColumnsForMultiple; - /** - * Meta column names - */ - private String[] metaColumnNames; - /** - * duplicateColMapping - */ - private int[][] duplicateColMapping; - private ExecutorService exec; - /** - * threadStatusObserver - */ - private ThreadStatusObserver threadStatusObserver; - /** - * CarbonCSVBasedDimSurrogateKeyGen - */ - private CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen; - - private DataType[] msrDataType; - /** - * wrapper object having the columnSchemaDetails - */ - private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper; - - /** - * to check whether column is a no dicitonary column or not - */ - private boolean[] isNoDictionaryColumn; - /** - * to check whether column is a no dicitonary column or not - */ - private boolean[] isStringDataType; - /** - * to check whether column is a no dicitonary column or not - */ - private String[] dataTypes; - - /** - * to check whether column is complex type column or not - */ - private boolean[] isComplexTypeColumn; - - /** - * to store index of no dictionapry column - */ - private int[] noDictionaryAndComplexIndexMapping; - - private GenericDataType[] complexTypes; - - private DirectDictionaryGenerator[] directDictionaryGenerators; - /** - * dimension column ids - */ - private String[] dimensionColumnIds; - - private Trans dis; - - /** - * Constructor - * - * @param s - * @param stepDataInterface - * @param c - * @param t - * @param dis - */ - public CarbonCSVBasedSeqGenStep(StepMeta s, StepDataInterface stepDataInterface, int c, - TransMeta t, Trans dis) { - super(s, stepDataInterface, c, t, dis); - csvFilepath = dis.getVariable("csvInputFilePath"); - this.dis = dis; - - } - - /** - * processRow - */ - public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException { - - try { - meta = (CarbonCSVBasedSeqGenMeta) smi; - StandardLogService.setThreadName(meta.getPartitionID(), null); - data = (CarbonCSVBasedSeqGenData) sdi; - - Object[] r = getRow(); // get row, blocks when needed! - if (first) { - CarbonTimeStatisticsFactory.getLoadStatisticsInstance() - .recordGeneratingDictionaryValuesTime(meta.getPartitionID(), - System.currentTimeMillis()); - first = false; - meta.initialize(); - final Object dataProcessingLockObject = CarbonDataProcessorManager.getInstance() - .getDataProcessingLockObject(meta.getDatabaseName() + '_' + meta.getTableName()); - synchronized (dataProcessingLockObject) { - // observer of writing file in thread - this.threadStatusObserver = new ThreadStatusObserver(); - if (csvFilepath == null) { - // isDBFactLoad = true; - csvFilepath = meta.getTableName(); - } - - if (null == measureCol) { - measureCol = Arrays.asList(meta.measureColumn); - } - // Update the Null value comparer and update the String against which we need - // to check the values coming from the previous step. - logCounter = - Integer.parseInt(CarbonCommonConstants.DATA_LOAD_LOG_COUNTER_DEFAULT_COUNTER); - if (null != getInputRowMeta()) { - meta.updateHierMappings(getInputRowMeta()); - populateCarbonMeasures(meta.measureColumn); - meta.msrMapping = getMeasureOriginalIndexes(meta.measureColumn); - - meta.memberMapping = getMemberMappingOriginalIndexes(); - - data.setInputSize(getInputRowMeta().size()); - - updatePropMap(meta.actualDimArray); - if (meta.isAggregate()) { - presentColumnMapIndex = createPresentColumnMapIndexForAggregate(); - } else { - presentColumnMapIndex = createPresentColumnMapIndex(); - - } - measurePresentMapping = createMeasureMappigs(measureCol); - measureSurrogateReqMapping = createMeasureSurrogateReqMapping(); - createForeignKeyMappingColumns(); - metaColumnNames = createColumnArrayFromMeta(); - } - - if (!meta.isAggregate()) { - updateHierarchyKeyGenerators(data.getKeyGenerators(), meta.hirches, meta.dimLens, - meta.dimColNames); - } - - data.setGenerator( - KeyGeneratorFactory.getKeyGenerator(getUpdatedLens(meta.dimLens, meta.dimPresent))); - - if (null != getInputRowMeta()) { - data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone()); - } - this.dimensionColumnIds = meta.getDimensionColumnIds(); - ColumnsInfo columnsInfo = new ColumnsInfo(); - columnsInfo.setDims(meta.dims); - columnsInfo.setDimColNames(meta.dimColNames); - columnsInfo.setKeyGenerators(data.getKeyGenerators()); - columnsInfo.setDatabaseName(meta.getDatabaseName()); - columnsInfo.setTableName(meta.getTableName()); - columnsInfo.setHierTables(meta.hirches.keySet()); - columnsInfo.setBatchSize(meta.getBatchSize()); - columnsInfo.setStoreType(meta.getStoreType()); - columnsInfo.setAggregateLoad(meta.isAggregate()); - columnsInfo.setMaxKeys(meta.dimLens); - columnsInfo.setPropColumns(meta.getPropertiesColumns()); - columnsInfo.setPropIndx(meta.getPropertiesIndices()); - columnsInfo.setTimeOrdinalCols(meta.timeOrdinalCols); - columnsInfo.setPropTypes(meta.getPropTypes()); - columnsInfo.setTimDimIndex(meta.timeDimeIndex); - columnsInfo.setDimHierRel(meta.getDimTableArray()); - columnsInfo.setBaseStoreLocation(getCarbonLocalBaseStoreLocation()); - columnsInfo.setTableName(meta.getTableName()); - columnsInfo.setPrimaryKeyMap(meta.getPrimaryKeyMap()); - columnsInfo.setMeasureColumns(meta.measureColumn); - columnsInfo.setComplexTypesMap(meta.getComplexTypes()); - columnsInfo.setDimensionColumnIds(this.dimensionColumnIds); - columnsInfo.setColumnSchemaDetailsWrapper(meta.getColumnSchemaDetailsWrapper()); - columnsInfo.setColumnProperties(meta.getColumnPropertiesMap()); - updateBagLogFileName(); - columnsInfo.setTimeOrdinalIndices(meta.timeOrdinalIndices); - surrogateKeyGen = new FileStoreSurrogateKeyGenForCSV(columnsInfo, meta.getPartitionID(), - meta.getSegmentId(), meta.getTaskNo()); - data.setSurrogateKeyGen(surrogateKeyGen); - updateStoreLocation(); - - // Check the insert hierarchies required or not based on that - // Create the list which will hold the hierarchies required to be created - // i.e. denormalized hierarchies. - if (null != getInputRowMeta()) { - denormHierarchies = getDenormalizedHierarchies(); - } - - if (null != getInputRowMeta()) { - // We consider that there is no time dimension,in these case - // the - // timeIndex = -1 - - ValueMetaInterface[] out = null; - out = new ValueMetaInterface[meta.normLength + meta.msrMapping.length]; - int outCounter = 0; - for (int i = 0; i < meta.actualDimArray.length; i++) { - if (meta.dimPresent[i]) { - ValueMetaInterface x = - new ValueMeta(meta.actualDimArray[i], ValueMetaInterface.TYPE_STRING, - ValueMetaInterface.STORAGE_TYPE_BINARY_STRING); - x.setStorageMetadata( - (new ValueMeta(meta.actualDimArray[i], ValueMetaInterface.TYPE_STRING, - ValueMetaInterface.STORAGE_TYPE_NORMAL))); - x.setStringEncoding(BYTE_ENCODING); - x.setStringEncoding(BYTE_ENCODING); - x.getStorageMetadata().setStringEncoding(BYTE_ENCODING); - - out[outCounter] = x; - outCounter++; - } - } - - for (int j = 0; j < meta.measureColumn.length; j++) { - for (int k = 0; k < data.getOutputRowMeta().size(); k++) { - if (meta.measureColumn[j] - .equalsIgnoreCase(data.getOutputRowMeta().getValueMeta(k).getName())) { - out[outCounter] = - new ValueMeta(meta.measureColumn[j], ValueMetaInterface.TYPE_NUMBER, - ValueMetaInterface.STORAGE_TYPE_NORMAL); - out[outCounter].setStorageMetadata( - new ValueMeta(meta.measureColumn[j], ValueMetaInterface.TYPE_NUMBER, - ValueMetaInterface.STORAGE_TYPE_NORMAL)); - outCounter++; - break; - } - } - } - data.getOutputRowMeta().setValueMetaList(Arrays.asList(out)); - } - } - columnSchemaDetailsWrapper = meta.getColumnSchemaDetailsWrapper(); - if (null != getInputRowMeta()) { - generateNoDictionaryAndComplexIndexMapping(); - data.getSurrogateKeyGen() - .setDimensionOrdinalToDimensionMapping(populateNameToCarbonDimensionMap()); - } - serializationNullFormat = - meta.getTableOptionWrapper().get(SERIALIZATION_NULL_FORMAT.getName()); - boolean badRecordsLoggerEnable; - boolean badRecordsLogRedirect = false; - boolean badRecordConvertNullDisable = false; - boolean isDataLoadFail = false; - badRecordsLoggerEnable = Boolean - .parseBoolean(meta.getTableOptionWrapper().get(BAD_RECORDS_LOGGER_ENABLE.getName())); - String bad_records_action = - meta.getTableOptionWrapper().get(BAD_RECORDS_ACTION.getName()); - if (null != bad_records_action) { - LoggerAction loggerAction = null; - try { - loggerAction = LoggerAction.valueOf(bad_records_action.toUpperCase()); - } catch (IllegalArgumentException e) { - loggerAction = LoggerAction.FORCE; - } - switch (loggerAction) { - case FORCE: - badRecordConvertNullDisable = false; - break; - case REDIRECT: - badRecordsLogRedirect = true; - badRecordConvertNullDisable = true; - break; - case IGNORE: - badRecordsLogRedirect = false; - badRecordConvertNullDisable = true; - break; - case FAIL: - isDataLoadFail = true; - break; - } - } - String key = meta.getDatabaseName() + '/' + meta.getTableName() + - '_' + meta.getTableName(); - badRecordsLogger = new BadRecordsLogger(key, csvFilepath, getBadLogStoreLocation( - meta.getDatabaseName() + '/' + meta.getTableName() + "/" + meta.getTaskNo()), - badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, - isDataLoadFail); - HashMap dateformatsHashMap = new HashMap(); - if (meta.dateFormat != null) { - String[] dateformats = meta.dateFormat.split(CarbonCommonConstants.COMMA); - for (String dateFormat:dateformats) { - String[] dateFormatSplits = dateFormat.split(":", 2); - dateformatsHashMap.put(dateFormatSplits[0].toLowerCase().trim(), - dateFormatSplits[1].trim()); - } - } - String[] DimensionColumnIds = meta.getDimensionColumnIds(); - directDictionaryGenerators = - new DirectDictionaryGenerator[DimensionColumnIds.length]; - for (int i = 0; i < DimensionColumnIds.length; i++) { - ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get( - DimensionColumnIds[i]); - if (columnSchemaDetails.isDirectDictionary()) { - String columnName = columnSchemaDetails.getColumnName(); - DataType columnType = columnSchemaDetails.getColumnType(); - if (dateformatsHashMap.containsKey(columnName)) { - directDictionaryGenerators[i] = - DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator( - columnType, dateformatsHashMap.get(columnName)); - } else { - directDictionaryGenerators[i] = - DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(columnType); - } - } - } - } - // no more input to be expected... - if (r == null) { - return processWhenRowIsNull(); - } - // proecess the first - Object[] out = process(r); - readCounter++; - if (null != out) { - writeCounter++; - putRow(data.getOutputRowMeta(), out); - } - // start multi-thread to process - int numberOfNodes; - try { - numberOfNodes = Integer.parseInt(CarbonProperties.getInstance() - .getProperty(CarbonCommonConstants.NUM_CORES_LOADING, - CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)); - } catch (NumberFormatException exc) { - numberOfNodes = NUM_CORES_DEFAULT_VAL; - } - - startReadingProcess(numberOfNodes); - badRecordsLogger.closeStreams(); - if (!meta.isAggregate()) { - closeNormalizedHierFiles(); - } - if (writeCounter == 0) { - return processWhenRowIsNull(); - } - CarbonUtil.writeLevelCardinalityFile(loadFolderLoc, meta.getTableName(), - getUpdatedCardinality()); - LOGGER.info("Record Procerssed For table: " + meta.getTableName()); - String logMessage = - "Summary: Carbon CSV Based Seq Gen Step : " + readCounter + ": Write: " + writeCounter; - LOGGER.info(logMessage); - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordGeneratingDictionaryValuesTime( - meta.getPartitionID(), System.currentTimeMillis()); - setOutputDone(); - - } catch (RuntimeException ex) { - LOGGER.error(ex); - throw ex; - } catch (Exception ex) { - LOGGER.error(ex); - throw new RuntimeException(ex); - } - return false; - } - - private void generateNoDictionaryAndComplexIndexMapping() { - isNoDictionaryColumn = new boolean[metaColumnNames.length]; - isComplexTypeColumn = new boolean[metaColumnNames.length]; - noDictionaryAndComplexIndexMapping = new int[metaColumnNames.length]; - isStringDataType = new boolean[metaColumnNames.length]; - dataTypes = new String[metaColumnNames.length]; - complexTypes = new GenericDataType[meta.getComplexTypeColumns().length]; - for (int i = 0; i < meta.noDictionaryCols.length; i++) { - for (int j = 0; j < metaColumnNames.length; j++) { - if (CarbonCommonConstants.STRING - .equalsIgnoreCase(meta.dimColDataTypes.get(metaColumnNames[j]))) { - isStringDataType[j] = true; - } - dataTypes[j] = meta.dimColDataTypes.get(metaColumnNames[j].toLowerCase()); - if (meta.noDictionaryCols[i].equalsIgnoreCase( - meta.getTableName() + CarbonCommonConstants.UNDERSCORE + metaColumnNames[j])) { - isNoDictionaryColumn[j] = true; - noDictionaryAndComplexIndexMapping[j] = i; - break; - } - } - } - for (int i = 0; i < meta.getComplexTypeColumns().length; i++) { - for (int j = 0; j < metaColumnNames.length; j++) { - if (meta.getComplexTypeColumns()[i].equalsIgnoreCase(metaColumnNames[j])) { - isComplexTypeColumn[j] = true; - complexTypes[i] = meta.complexTypes.get(meta.getComplexTypeColumns()[i]); - noDictionaryAndComplexIndexMapping[j] = i + meta.noDictionaryCols.length; - break; - } - } - } - } - - private void startReadingProcess(int numberOfNodes) throws KettleException, InterruptedException { - startProcess(numberOfNodes); - } - - private boolean processWhenRowIsNull() throws KettleException { - // If first request itself is null then It will not enter the first block and - // in data surrogatekeygen will not be initialized so it can throw NPE. - if (data.getSurrogateKeyGen() == null) { - setOutputDone(); - LOGGER.info("Record Procerssed For table: " + meta.getTableName()); - String logMessage = - "Summary: Carbon CSV Based Seq Gen Step: Read: " + readCounter + ": Write: " - + writeCounter; - LOGGER.info(logMessage); - return false; - } - - setOutputDone(); - LOGGER.info("Record Processed For table: " + meta.getTableName()); - String logMessage = - "Summary: Carbon CSV Based Seq Gen Step: Read: " + readCounter + ": Write: " - + writeCounter; - LOGGER.info(logMessage); - return false; - } - - /** - * holds the value to be considered as null while dataload - */ - private String serializationNullFormat; - - private List getDenormalizedHierarchies() { - List hierList = Arrays.asList(meta.hierNames); - List denormHiers = new ArrayList(10); - for (Iterator> iterator = meta.hirches.entrySet().iterator(); iterator - .hasNext(); ) { - Entry entry = iterator.next(); - String name = entry.getKey(); - - if (hierList.contains(name)) { - continue; - } else if (entry.getValue().length > 1) { - denormHiers.add(name); - } - } - - return denormHiers; - } - - private void updatePropMap(String[] actualDimArray) { - if (null == propMap) { - propMap = new HashMap(actualDimArray.length); - } - List currentColNames = new ArrayList(10); - for (int i = 0; i < getInputRowMeta().size(); i++) { - currentColNames.add(getInputRowMeta().getValueMeta(i).getName()); - } - - List currentColName = new ArrayList(actualDimArray.length); - - for (int i = 0; i < getInputRowMeta().size(); i++) { - String columnName = getInputRowMeta().getValueMeta(i).getName(); - String hier = meta.foreignKeyHierarchyMap.get(columnName); - if (null != hier) { - if (hier.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) { - String[] splittedHiers = hier.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - for (String hierName : splittedHiers) { - String tableName = meta.getHierDimTableMap().get(hier); - String[] cols = meta.hierColumnMap.get(hierName); - if (null != cols) { - for (String column : cols) { - currentColName.add(tableName + '_' + column); - } - } - } - } else { - String tableName = meta.getHierDimTableMap().get(hier); - - String[] columns = meta.hierColumnMap.get(hier); - - if (null != columns) { - for (String column : columns) { - currentColName.add(tableName + '_' + column); - } - } - } - } else - // then it can be direct column name if not foreign key. - { - currentColName.add(meta.getTableName() + '_' + columnName); - } - } - - String[] currentColNamesArray = currentColName.toArray(new String[currentColName.size()]); - - List metahierVoList = meta.getMetahierVoList(); - - if (null == metahierVoList) { - return; - } - for (HierarchiesInfo hierInfo : metahierVoList) { - - Map columnPropMap = hierInfo.getColumnPropMap(); - - Set> entrySet = columnPropMap.entrySet(); - - for (Entry entry : entrySet) { - String[] propColmns = entry.getValue(); - int[] index = getIndex(currentColNamesArray, propColmns); - propMap.put(entry.getKey(), index); - } - } - - } - - private int[] getIndex(String[] currentColNamesArray, String[] propColmns) { - int[] resultIndex = new int[propColmns.length]; - - for (int i = 0; i < propColmns.length; i++) { - for (int j = 0; j < currentColNamesArray.length; j++) { - if (propColmns[i].equalsIgnoreCase(currentColNamesArray[j])) { - resultIndex[i] = j; - break; - } - } - } - - return resultIndex; - } - - private void closeNormalizedHierFiles() throws KettleException { - if (null == filemanager) { - return; - } - int hierLen = filemanager.size(); - - for (int i = 0; i < hierLen; i++) { - FileData hierFileData = (FileData) filemanager.get(i); - String hierInProgressFileName = hierFileData.getFileName(); - HierarchyValueWriterForCSV hierarchyValueWriter = - nrmlizedHierWriterMap.get(hierInProgressFileName); - if (null == hierarchyValueWriter) { - continue; - } - - List holders = hierarchyValueWriter.getByteArrayList(); - Collections.sort(holders); - - for (ByteArrayHolder holder : holders) { - hierarchyValueWriter.writeIntoHierarchyFile(holder.getMdKey(), holder.getPrimaryKey()); - } - - // now write the byte array in the file. - FileChannel bufferedOutStream = hierarchyValueWriter.getBufferedOutStream(); - if (null == bufferedOutStream) { - continue; - } - CarbonUtil.closeStreams(bufferedOutStream); - - hierInProgressFileName = hierFileData.getFileName(); - int counter = hierarchyValueWriter.getCounter(); - String storePath = hierFileData.getStorePath(); - String changedFileName = hierInProgressFileName + (counter - 1); - hierInProgressFileName = changedFileName + CarbonCommonConstants.FILE_INPROGRESS_STATUS; - - File currentFile = new File(storePath + File.separator + hierInProgressFileName); - File destFile = new File(storePath + File.separator + changedFileName); - if (currentFile.exists()) { - boolean renameTo = currentFile.renameTo(destFile); - - if (!renameTo) { - LOGGER.info("Not Able to Rename File : " + currentFile.getName()); - } - } - - } - - } - - /** - * Load Store location - */ - private void updateStoreLocation() { - loadFolderLoc = CarbonDataProcessorUtil - .getLocalDataFolderLocation(meta.getDatabaseName(), meta.getTableName(), meta.getTaskNo(), - meta.getPartitionID(), meta.getSegmentId() + "", false); - } - - private String getBadLogStoreLocation(String storeLocation) { - String badLogStoreLocation = - CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC); - badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation; - - return badLogStoreLocation; - } - - private void updateBagLogFileName() { - csvFilepath = new File(csvFilepath).getName(); - if (csvFilepath.indexOf(".") > -1) { - csvFilepath = csvFilepath.substring(0, csvFilepath.indexOf(".")); - } - - csvFilepath = csvFilepath + '_' + System.currentTimeMillis(); - - } - - private void startProcess(final int numberOfNodes) throws RuntimeException { - exec = Executors.newFixedThreadPool(numberOfNodes); - - Callable callable = new Callable() { - @Override public Void call() throws RuntimeException { - StandardLogService - .setThreadName(StandardLogService.getPartitionID(meta.getTableName()), null); - try { - doProcess(); - } catch (Throwable e) { - LOGGER.error(e, "Thread is terminated due to error"); - threadStatusObserver.notifyFailed(e); - } - return null; - } - }; - List> results = new ArrayList>(10); - for (int i = 0; i < numberOfNodes; i++) { - results.add(exec.submit(callable)); - } - - this.resultArray = results.toArray(new Future[results.size()]); - try { - for (int j = 0; j < this.resultArray.length; j++) { - this.resultArray[j].get(); - } - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException("Thread InterruptedException", e); - } finally { - exec.shutdownNow(); - } - } - - private int[] getUpdatedLens(int[] lens, boolean[] presentDims) { - int k = 0; - int[] integers = new int[meta.normLength]; - for (int i = 0; i < lens.length; i++) { - if (presentDims[i]) { - integers[k] = lens[i]; - k++; - } - } - return integers; - } - - /** - * @return - */ - private int[] getUpdatedCardinality() { - int[] maxSurrogateKeyArray = data.getSurrogateKeyGen().max; - - List dimCardWithComplex = new ArrayList(); - - for (int i = 0; i < meta.dimColNames.length; i++) { - GenericDataType complexDataType = - meta.complexTypes.get(meta.dimColNames[i].substring(meta.getTableName().length() + 1)); - if (complexDataType != null) { - complexDataType.fillCardinalityAfterDataLoad(dimCardWithComplex, maxSurrogateKeyArray); - } else { - dimCardWithComplex.add(maxSurrogateKeyArray[i]); - } - } - - int[] complexDimCardinality = new int[dimCardWithComplex.size()]; - for (int i = 0; i < dimCardWithComplex.size(); i++) { - complexDimCardinality[i] = dimCardWithComplex.get(i); - } - return complexDimCardinality; - } - - private void doProcess() throws RuntimeException { - try { - for (DirectDictionaryGenerator directDictionaryGenerator: directDictionaryGenerators) { - if (directDictionaryGenerator != null) { - directDictionaryGenerator.initialize(); - } - } - - while (true) { - Object[] r = null; - synchronized (getRowLock) { - - r = getRow(); - readCounter++; - } - - // no more input to be expected... - if (r == null) { - readCounter--; - break; - } - Object[] out = process(r); - if (null == out) { - continue; - } - - synchronized (putRowLock) { - putRow(data.getOutputRowMeta(), out); - processRecord(); - writeCounter++; - } - } - } catch (RuntimeException e) { - throw e; - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - private void processRecord() { - if (readCounter % logCounter == 0) { - LOGGER.info("Record Procerssed For table: " + meta.getTableName()); - String logMessage = "Carbon Csv Based Seq Gen Step: Record Read : " + readCounter; - LOGGER.info(logMessage); - } - } - - private String getCarbonLocalBaseStoreLocation() { - String tempLocationKey = - meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName() - + CarbonCommonConstants.UNDERSCORE + meta.getTaskNo(); - String strLoc = CarbonProperties.getInstance() - .getProperty(tempLocationKey, CarbonCommonConstants.STORE_LOCATION_DEFAULT_VAL); - File f = new File(strLoc); - String absoluteStorePath = f.getAbsolutePath(); - return absoluteStorePath; - } - - private Object[] process(Object[] r) throws RuntimeException { - try { - Object[] out = populateOutputRow(r); - if (out != null) { - for (int i = 0; i < meta.normLength - meta.complexTypes.size(); i++) { - if (null == NonDictionaryUtil.getDimension(i, out)) { - NonDictionaryUtil.setDimension(i, 1, out); - } - } - } - return out; - - } catch (KettleException e) { - throw new RuntimeException(e); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - private Object[] populateOutputRow(Object[] r) throws KettleException { - - // Copy the dimension String values to output - int[] memberMapping = meta.memberMapping; - int inputColumnsSize = metaColumnNames.length; - boolean isGenerated = false; - int generatedSurrogate = -1; - - //If CSV Exported from DB and we enter one row down then that row become empty. - // In that case it will have first value empty and other values will be null - // So If records is coming like this then we need to write this records as a bad Record. - - if (null == r[0] && badRecordsLogger.isBadRecordConvertNullDisable()) { - badRecordsLogger - .addBadRecordsToBuilder(r, "Column Names are coming NULL"); - return null; - } - - Map dictionaryCaches = surrogateKeyGen.getDictionaryCaches(); - Object[] out = - new Object[meta.normLength + meta.msrs.length - meta.complexTypes.size()]; - int dimLen = meta.dims.length; - - Object[] newArray = new Object[CarbonCommonConstants.ARRAYSIZE]; - - ByteBuffer[] byteBufferArr = null; - if (null != meta.noDictionaryCols) { - byteBufferArr = new ByteBuffer[meta.noDictionaryCols.length + meta.complexTypes.size()]; - } - int i = 0; - int index = 0; - int l = 0; - int msrCount = 0; - boolean isNull = false; - for (int j = 0; j < inputColumnsSize; j++) { - String columnName = metaColumnNames[j]; - String foreignKeyColumnName = foreignKeyMappingColumns[j]; - // check if it is ignore dictionary dimension or not . if yes directly write byte buffer - String tuple = null == r[j] ? - CarbonCommonConstants.MEMBER_DEFAULT_VAL : - (String) r[j]; - // check whether the column value is the value to be serialized as null. - boolean isSerialized = false; - if (tuple.equalsIgnoreCase(serializationNullFormat)) { - tuple = CarbonCommonConstants.MEMBER_DEFAULT_VAL; - isSerialized = true; - } - if (isNoDictionaryColumn[j]) { - String dimensionValue = - processnoDictionaryDim(noDictionaryAndComplexIndexMapping[j], tuple, dataTypes[j], - isStringDataType[j], byteBufferArr); - if (!isSerialized && !isStringDataType[j] && CarbonCommonConstants.MEMBER_DEFAULT_VAL - .equals(dimensionValue)) { - failDataLoad(r, index, columnName, msrDataType[meta.msrMapping[msrCount]].name()); - addEntryToBadRecords(r, j, columnName, dataTypes[j]); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - } - continue; - } - // There is a possibility that measure can be referred as dimensions also - // so in that case we need to just copy the value into the measure column index. - //if it enters here means 3 possibility - //1) this is not foreign key it can be direct columns - //2) This column present in the csv file but in the schema it is not present. - //3) This column can be measure column - - if (measurePresentMapping[j]) { - String msr = tuple == null ? null : tuple.toString(); - isNull = CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(msr); - if (measureSurrogateReqMapping[j] && !isNull) { - Integer surrogate = 0; - if (null == foreignKeyColumnName) { - // If foreignKeyColumnName is null till here that means this - // measure column is of type count and data type may be string - // so we have to create the surrogate key for the values. - surrogate = createSurrogateForMeasure(msr, columnName); - if (presentColumnMapIndex[j] > -1) { - isGenerated = true; - generatedSurrogate = surrogate; - } - } else { - surrogate = surrogateKeyGen.generateSurrogateKeys(msr, foreignKeyColumnName); - } - - out[memberMapping[dimLen + index]] = surrogate.doubleValue(); - } else if (!isSerialized && (isNull || msr == null - || msr.length() == 0)) { - failDataLoad(r, index, columnName, - msrDataType[meta.msrMapping[msrCount]].name()); - addEntryToBadRecords(r, j, columnName, - msrDataType[meta.msrMapping[msrCount]].name()); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - } else { - try { - if (!isNull && null != msr && msr.length() > 0) { - Object measureValueBasedOnDataType = DataTypeUtil - .getMeasureValueBasedOnDataType(msr, msrDataType[meta.msrMapping[msrCount]], - meta.carbonMeasures[meta.msrMapping[msrCount]]); - if (null == measureValueBasedOnDataType) { - addEntryToBadRecords(r, j, columnName, - msrDataType[meta.msrMapping[msrCount]].name()); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - LOGGER.warn("Cannot convert : " + msr - + " to Numeric type value. Value considered as null."); - } - out[memberMapping[dimLen + index] - meta.complexTypes.size()] = - measureValueBasedOnDataType; - } - } catch (NumberFormatException e) { - failDataLoad(r, index, columnName, - msrDataType[meta.msrMapping[msrCount]].name()); - addEntryToBadRecords(r, j, columnName, msrDataType[meta.msrMapping[msrCount]].name()); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - LOGGER.warn( - "Cannot convert : " + msr + " to Numeric type value. Value considered as null."); - out[memberMapping[dimLen + index] - meta.complexTypes.size()] = null; - } - } - - index++; - msrCount++; - if (presentColumnMapIndex[j] < 0 && null == foreignKeyColumnName) { - continue; - } - } - - boolean isPresentInSchema = false; - if (null == foreignKeyColumnName) { - //if it enters here means 3 possibility - //1) this is not foreign key it can be direct columns - //2) This column present in the csv file but in the schema it is not present. - //3) This column can be measure column - int m = presentColumnMapIndex[j]; - if (m >= 0) { - isPresentInSchema = true; - } - - if (isPresentInSchema) { - foreignKeyColumnName = meta.dimColNames[m]; - } else { - continue; - } - } - - //If it refers to multiple hierarchy by same foreign key - if (foreignKeyMappingColumnsForMultiple[j] != null) { - String[] splittedHiers = foreignKeyMappingColumnsForMultiple[j]; - - for (String hierForignKey : splittedHiers) { - Dictionary dicCache = dictionaryCaches.get(hierForignKey); - - String actualHierName = null; - if (!isPresentInSchema) { - actualHierName = meta.hierNames[l++]; - - } - - Map cache = surrogateKeyGen.getHierCache().get(actualHierName); - int[] surrogateKeyForHierarchy = null; - if (null != cache) { - - Integer keyFromCsv = dicCache.getSurrogateKey(tuple); - - if (null != keyFromCsv) { - surrogateKeyForHierarchy = cache.get(keyFromCsv); - } else { - addMemberNotExistEntry(r, j, columnName); - return null; - } - // If cardinality exceeded for some levels then - // for that hierarchy will not be their - // so while joining with fact table if we are - // getting this scenerio we will log it - // in bad records - if (null == surrogateKeyForHierarchy) { - addEntryToBadRecords(r, j, columnName); - return null; - - } - } else { - surrogateKeyForHierarchy = new int[1]; - surrogateKeyForHierarchy[0] = - surrogateKeyGen.generateSurrogateKeys(tuple, foreignKeyColumnName); - } - for (int k = 0; k < surrogateKeyForHierarchy.length; k++) { - if (dimPresentCsvOrder[i]) { - out[memberMapping[i]] = surrogateKeyForHierarchy[k]; - } - - i++; - } - - } - - } else if (isComplexTypeColumn[j]) { - //If it refers to single hierarchy - try { - GenericDataType complexType = - complexTypes[noDictionaryAndComplexIndexMapping[j] - meta.noDictionaryCols.length]; - ByteArrayOutputStream byteArray = new ByteArrayOutputStream(); - DataOutputStream dataOutputStream = new DataOutputStream(byteArray); - complexType.parseStringAndWriteByteArray(meta.getTableName(), tuple, - new String[] { meta.getComplexDelimiterLevel1(), meta.getComplexDelimiterLevel2() }, - 0, dataOutputStream, surrogateKeyGen); - byteBufferArr[noDictionaryAndComplexIndexMapping[j]] = - ByteBuffer.wrap(byteArray.toByteArray()); - if (null != byteArray) { - byteArray.close(); - } - } catch (IOException e1) { - throw new KettleException( - "Parsing complex string and generating surrogates/ByteArray failed. ", e1); - } - i++; - } else { - Dictionary dicCache = dictionaryCaches.get(foreignKeyColumnName); - - String actualHierName = null; - if (!isPresentInSchema) { - actualHierName = meta.hierNames[l++]; - - } - - Map cache = surrogateKeyGen.getHierCache().get(actualHierName); - int[] surrogateKeyForHrrchy = null; - if (null != cache) { - Integer keyFromCsv = dicCache.getSurrogateKey(tuple); - - if (null != keyFromCsv) { - surrogateKeyForHrrchy = cache.get(keyFromCsv); - } else { - addMemberNotExistEntry(r, j, columnName); - return null; - } - // If cardinality exceeded for some levels then for that hierarchy will not be their - // so while joining with fact table if we are getting this scenerio we will log it - // in bad records - if (null == surrogateKeyForHrrchy) { - addEntryToBadRecords(r, j, columnName); - return null; - - } - } else { - int[] propIndex = propMap.get(foreignKeyColumnName); - Object[] properties; - if (null == propIndex) { - properties = new Object[0]; - } else { - properties = new Object[propIndex.length]; - for (int ind = 0; ind < propIndex.length; ind++) { - Object objectValue = r[propIndex[ind]]; - properties[ind] = null == objectValue ? - CarbonCommonConstants.MEMBER_DEFAULT_VAL : (String)objectValue; - } - } - surrogateKeyForHrrchy = new int[1]; - if (isGenerated && !isNull) { - surrogateKeyForHrrchy[0] = generatedSurrogate; - isGenerated = false; - generatedSurrogate = -1; - } else { - int m = j; - if (isPresentInSchema) { - m = presentColumnMapIndex[j]; - } - ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimensionColumnIds[m]); - if (details.isDirectDictionary()) { - surrogateKeyForHrrchy[0] = - directDictionaryGenerators[m].generateDirectSurrogateKey(tuple); - if (!isSerialized && surrogateKeyForHrrchy[0] == 1) { - failDataLoad(r, index, columnName, details.getColumnType().getName()); - addEntryToBadRecords(r, j, columnName, details.getColumnType().name()); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - } - surrogateKeyGen.max[m] = Integer.MAX_VALUE; - - } else { - String parsedValue = DataTypeUtil.parseValue(tuple, data.getSurrogateKeyGen() - .getDimensionOrdinalToDimensionMapping()[memberMapping[i]]); - if (null == parsedValue) { - surrogateKeyForHrrchy[0] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - } else { - surrogateKeyForHrrchy[0] = - surrogateKeyGen.generateSurrogateKeys(parsedValue, foreignKeyColumnName); - } - } - } - if (surrogateKeyForHrrchy[0] == CarbonCommonConstants.INVALID_SURROGATE_KEY) { - - if (!isSerialized) { - int m = j; - if (isPresentInSchema) { - m = presentColumnMapIndex[j]; - } - ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimensionColumnIds[m]); - failDataLoad(r, index, columnName, details.getColumnType().getName()); - addEntryToBadRecords(r, j, columnName); - if (badRecordsLogger.isBadRecordConvertNullDisable()) { - return null; - } - } - surrogateKeyForHrrchy[0] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY; - } - } - for (int k = 0; k < surrogateKeyForHrrchy.length; k++) { - if (dimPresentCsvOrder[i]) { - if (duplicateColMapping[j] != null) { - for (int m = 0; m < duplicateColMapping[j].length; m++) { - out[duplicateColMapping[j][m]] = Integer.valueOf(surrogateKeyForHrrchy[k]); - } - } else { - out[memberMapping[i]] = Integer.valueOf(surrogateKeyForHrrchy[k]); - } - } - - i++; - } - } - } - - insertHierIfRequired(out); - NonDictionaryUtil - .prepareOut(newArray, byteBufferArr, out, dimLen - meta.complexTypes.size()); - - return newArray; - } - - private void failDataLoad(Object[] row, int index, String columnName, String dataType) - throws KettleException { - if (badRecordsLogger.isDataLoadFail()) { - String errorMessage = getBadRecordEntry(row, index, columnName, dataType); - dis.setVariable(CarbonCommonConstants.BAD_RECORD_KEY, errorMessage); - LOGGER.error("Data load failed due to bad record. " + errorMessage); - throw new KettleException("Data load failed due to bad record"); - } - } - - private void addEntryToBadRecords(Object[] r, int j, String columnName, String dataType) { - dataType = DataTypeUtil.getColumnDataTypeDisplayName(dataType); - badRecordsLogger.addBadRecordsToBuilder(r, - "The value " + " \"" + r[j] + "\"" + " with column name " + columnName - + " and column data type " + dataType + " is not a valid " + dataType + " type."); - } - - private String getBadRecordEntry(Object[] r, int j, String columnName, String dataType) { - dataType = DataTypeUtil.getColumnDataTypeDisplayName(dataType); - String badRecord = "The value " + " \"" + r[j] + "\"" + " with column name " + columnName - + " and column data type " + dataType + " is not a valid Record"; - return badRecord; - } - - private void addEntryToBadRecords(Object[] r, int j, String columnName) { - badRecordsLogger.addBadRecordsToBuilder(r, - "Surrogate key for value " + " \"" + r[j] + "\"" + " with column name " + columnName - + " not found in dictionary cache"); - } - - private void addMemberNotExistEntry(Object[] r, int j, String columnName) { - badRecordsLogger.addBadRecordsToBuilder(r, - "For Coulmn " + columnName + " \"" + r[j] + "\"" - + " member not exist in the dimension table "); - } - - private void insertHierIfRequired(Object[] out) throws KettleException { - if (denormHierarchies.size() > 0) { - insertHierarichies(out); - } - } - - private int[] createPresentColumnMapIndex() { - int[] presentColumnMapIndex = new int[getInputRowMeta().size()]; - duplicateColMapping = new int[getInputRowMeta().size()][]; - Arrays.fill(presentColumnMapIndex, -1); - for (int j = 0; j < getInputRowMeta().size(); j++) { - String columnName = getInputRowMeta().getValueMeta(j).getName(); - - int m = 0; - - String foreignKey = meta.foreignKeyHierarchyMap.get(columnName); - if (foreignKey == null) { - List repeats = new ArrayList(10); - for (String col : meta.dimColNames) { - if (col.equalsIgnoreCase(meta.getTableName() + '_' + columnName)) { - presentColumnMapIndex[j] = m; - repeats.add(m); - } - m++; - } - if (repeats.size() > 1) { - int[] dims = new int[repeats.size()]; - for (int i = 0; i < dims.length; i++) { - dims[i] = repeats.get(i); - } - duplicateColMapping[j] = dims; - } - - } else { - for (String col : meta.actualDimArray) { - if (col.equalsIgnoreCase(columnName)) { - presentColumnMapIndex[j] = m; - break; - } - m++; - } - - } - } - return presentColumnMapIndex; - } - - private int[] createPresentColumnMapIndexForAggregate() { - int[] presentColumnMapIndex = new int[getInputRowMeta().size()]; - duplicateColMapping = new int[getInputRowMeta().size()][]; - Arrays.fill(presentColumnMapIndex, -1); - for (int j = 0; j < getInputRowMeta().size(); j++) { - String columnName = getInputRowMeta().getValueMeta(j).getName(); - - int m = 0; - - String foreignKey = meta.foreignKeyHierarchyMap.get(columnName); - if (foreignKey == null) { - for (String col : meta.actualDimArray) { - if (col.equalsIgnoreCase(columnName)) { - presentColumnMapIndex[j] = m; - break; - } - m++; - } - } - } - return presentColumnMapIndex; - } - - private String[] createColumnArrayFromMeta() { - String[] metaColumnNames = new String[getInputRowMeta().size()]; - for (int j = 0; j < getInputRowMeta().size(); j++) { - metaColumnNames[j] = getInputRowMeta().getValueMeta(j).getName(); - } - return metaColumnNames; - } - - private boolean[] createMeasureMappigs(List measureCol) { - int size = getInputRowMeta().size(); - boolean[] measurePresentMapping = new boolean[size]; - for (int j = 0; j < size; j++) { - String columnName = getInputRowMeta().getValueMeta(j).getName(); - for (String measure : measureCol) { - if (measure.equalsIgnoreCase(columnName)) { - measurePresentMapping[j] = true; - break; - } - } - } - return measurePresentMapping; - - } - - private boolean[] createMeasureSurrogateReqMapping() { - int size = getInputRowMeta().size(); - boolean[] measureSuurogateReqMapping = new boolean[size]; - for (int j = 0; j < size; j++) { - String columnName = getInputRowMeta().getValueMeta(j).getName(); - Boolean isPresent = meta.getMeasureSurrogateRequired().get(columnName); - if (null != isPresent && isPresent) { - measureSuurogateReqMapping[j] = true; - } - } - return measureSuurogateReqMapping; - } - - private void createForeignKeyMappingColumns() { - int size = getInputRowMeta().size(); - foreignKeyMappingColumns = new String[size]; - foreignKeyMappingColumnsForMultiple = new String[size][]; - for (int j = 0; j < size; j++) { - String columnName = getInputRowMeta().getValueMeta(j).getName(); - String foreignKeyColumnName = meta.foreignKeyPrimaryKeyMap.get(columnName); - if (foreignKeyColumnName != null) { - if (foreignKeyColumnName.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) { - String[] splittedHiers = - foreignKeyColumnName.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - foreignKeyMappingColumnsForMultiple[j] = splittedHiers; - foreignKeyMappingColumns[j] = foreignKeyColumnName; - } else { - foreignKeyMappingColumns[j] = foreignKeyColumnName; - } - } - } - } - - private int createSurrogateForMeasure(String member, String columnName) - throws KettleException { - String colName = meta.getTableName() + '_' + columnName; - return data.getSurrogateKeyGen().getSurrogateForMeasure(member, colName); - } - - private void insertHierarichies(Object[] rowWithKeys) throws KettleException { - - try { - for (String hierName : denormHierarchies) { - - String storeLocation = ""; - String hierInprogName = hierName + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION; - HierarchyValueWriterForCSV hierWriter = nrmlizedHierWriterMap.get(hierInprogName); - storeLocation = loadFolderLoc; - if (null == filemanager) { - filemanager = new FileManager(); - filemanager.setName(storeLocation); - } - if (null == hierWriter) { - FileData fileData = new FileData(hierInprogName, storeLocation); - hierWriter = new HierarchyValueWriterForCSV(hierInprogName, storeLocation); - filemanager.add(fileData); - nrmlizedHierWriterMap.put(hierInprogName, hierWriter); - } - - int[] levelsIndxs = meta.hirches.get(hierName); - int[] levelSKeys = new int[levelsIndxs.length]; - - if (meta.complexTypes.get(meta.hierColumnMap.get(hierName)[0]) == null) { - for (int i = 0; i < levelSKeys.length; i++) { - levelSKeys[i] = (Integer) rowWithKeys[levelsIndxs[i]]; - } - - if (levelSKeys.length > 1) { - data.getSurrogateKeyGen().checkNormalizedHierExists(levelSKeys, hierName, hierWriter); - } - } - } - } catch (Exception e) { - throw new KettleException(e.getMessage(), e); - } - } - - private boolean isMeasureColumn(String msName, boolean compareWithTable) { - String msrNameTemp; - for (String msrName : meta.measureColumn) { - msrNameTemp = msrName; - if (compareWithTable) { - msrNameTemp = meta.getTableName() + '_' + msrNameTemp; - } - if (msrNameTemp.equalsIgnoreCase(msName)) { - return true; - } - } - return false; - } - - private int[] getMeasureOriginalIndexes(String[] originalMsrCols) { - List currMsrCol = new ArrayList(10); - for (int i = 0; i < getInputRowMeta().size(); i++) { - String columnName = getInputRowMeta().getValueMeta(i).getName(); - for (String measureCol : originalMsrCols) { - if (measureCol.equalsIgnoreCase(columnName)) { - currMsrCol.add(columnName); - break; - } - } - } - String[] currentMsrCols = currMsrCol.toArray(new String[currMsrCol.size()]); - - int[] indexs = new int[currentMsrCols.length]; - - for (int i = 0; i < currentMsrCols.length; i++) { - for (int j = 0; j < originalMsrCols.length; j++) { - if (currentMsrCols[i].equalsIgnoreCase(originalMsrCols[j])) { - indexs[i] = j; - break; - } - } - } - - return indexs; - } - - private int[] getMemberMappingOriginalIndexes() { - int[] memIndexes = new int[meta.dimLens.length + meta.msrs.length]; - Arrays.fill(memIndexes, -1); - String actualColumnName = null; - List allColumnsNamesFromCSV = new ArrayList(10); - for (int i = 0; i < getInputRowMeta().size(); i++) { - allColumnsNamesFromCSV.add(getInputRowMeta().getValueMeta(i).getName()); - } - - List currentColName = new ArrayList(meta.actualDimArray.length); - List duplicateNames = new ArrayList(10); - for (int i = 0; i < getInputRowMeta().size(); i++) { - String columnName = getInputRowMeta().getValueMeta(i).getName(); - String hier = meta.foreignKeyHierarchyMap.get(columnName); - - String uniqueName = meta.getTableName() + '_' + columnName; - if (null != hier) { - - if (hier.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) { - getCurrenColForMultiHier(currentColName, hier); - } else { - String tableName = meta.getHierDimTableMap().get(hier); - - String[] columns = meta.hierColumnMap.get(hier); - - if (null != columns) { - for (String column : columns) { - //currentColumnNames[k++] = column; - currentColName.add(tableName + '_' + column); - } - } - } - - if (isMeasureColumn(columnName, false)) { - currentColName.add(uniqueName); - } - - } else // then it can be direct column name if not foreign key. - { - if (!meta.isAggregate()) { - currentColName.add(uniqueName); - //add to duplicate column list if it is a repeated column. it is required since the - // member mapping is 1 to 1 mapping - //of csv columns and schema columns. so if schema columns are repeated then we have to - // handle it in special way. - checkAndAddDuplicateCols(duplicateNames, uniqueName); - } else { - actualColumnName = meta.columnAndTableNameColumnMapForAggMap.get(columnName); - if (actualColumnName != null) { - currentColName.add(meta.columnAndTableNameColumnMapForAggMap.get(columnName)); - } else { - currentColName.add(uniqueName); - } - } - } - } - //Add the duplicate columns at the end so that it won't create any problem with current mapping. - currentColName.addAll(duplicateNames); - String[] currentColNamesArray = currentColName.toArray(new String[currentColName.size()]); - - // We will use same array for dimensions and measures - // First create the mapping for dimensions. - int dimIndex = 0; - Map counterMap = new HashMap(16); - // Setting dimPresent value in CSV order as we need it later - dimPresentCsvOrder = new boolean[meta.dimPresent.length]; - // var used to set measures value (in next loop) - int toAddInIndex = 0; - int tmpIndex = 0; - for (int i = 0; i < currentColNamesArray.length; i++) { - if (isMeasureColumn(currentColNamesArray[i], true) && isNotInDims(currentColNamesArray[i])) { - continue; - } - int n = 0; - for (int j = 0; j < meta.actualDimArray.length; j++) { - - if (currentColNamesArray[i].equalsIgnoreCase(meta.dimColNames[j])) { - - String mapKey = currentColNamesArray[i] + "__" + j; - if (null == counterMap.get(mapKey)) { - dimPresentCsvOrder[tmpIndex] = meta.dimPresent[j];//CHECKSTYLE:ON - tmpIndex++; - counterMap.put(mapKey, true); - if (!meta.dimPresent[j]) { - dimIndex++; - continue; - } - memIndexes[dimIndex++] = n; - // Added one more value to memIndexes, increase counter - toAddInIndex++; - break; - } else { - n++; - continue; - } - } - if (meta.dimPresent[j]) { - n++; - } - } - } - - for (int actDimLen = 0; actDimLen < meta.actualDimArray.length; actDimLen++) { - boolean found = false; - for (int csvHeadLen = 0; csvHeadLen < currentColNamesArray.length; csvHeadLen++) { - if (meta.dimColNames[actDimLen].equalsIgnoreCase(currentColNamesArray[csvHeadLen])) { - found = true; - break; - } - } - - if (!found) { - dimIndex++; - toAddInIndex++; - } - } - - // Now create the mapping of measures - // There may be case when measure column is present in the CSV file - // but not present in the schema , in that case we need to skip that column while - // sending the output to next step. - // Or Measure can be in any ordinal in the csv - - int k = 0; - Map existsMap = new HashMap(16); - - for (int i = 0; i < currentColNamesArray.length; i++) { - k = calculateMeasureOriginalIndexes(memIndexes, currentColNamesArray, dimIndex, toAddInIndex, - k, existsMap, i); - } - - return memIndexes; - } - - private void getCurrenColForMultiHier(List currentColName, String hier) { - String[] splittedHiers = hier.split(CarbonCommonConstants.COMA_SPC_CHARACTER); - for (String hierName : splittedHiers) { - String tableName = meta.getHierDimTableMap().get(hierName); - - String[] cols = meta.hierColumnMap.get(hierName); - if (null != cols) { - for (String column : cols) { - currentColName.add(tableName + '_' + column); - } - } - } - } - - private void checkAndAddDuplicateCols(List duplicateNames, String uniqueName) { - boolean exists = false; - for (int i = 0; i < meta.dimColNames.length; i++) { - if (uniqueName.equals(meta.dimColNames[i])) { - if (exists) { - duplicateNames.add(uniqueName); - } - exists = true; - } - } - } - - /** - * calculateMeasureOriginalIndexes - * - * @param memIndexes - * @param currentColNamesArray - * @param dimIndex - * @param toAddInIndex - * @param k - * @param existsMap - * @param i - * @return - */ - public int calculateMeasureOriginalIndexes(int[] memIndexes, String[] currentColNamesArray, - int dimIndex, int toAddInIndex, int k, Map existsMap, int i) { - for (int j = 0; j < meta.measureColumn.length; j++) { - if (currentColNamesArray[i] - .equalsIgnoreCase(meta.getTableName() + '_' + meta.measureColumn[j])) { - if (existsMap.get(meta.measureColumn[j]) == null) { - memIndexes[k + dimIndex] = toAddInIndex + j; - k++; - existsMap.put(meta.measureColumn[j], true); - break; - } - } - } - return k; - } - - private boolean isNotInDims(String columnName) { - for (String dimName : meta.dimColNames) { - if (dimName.equalsIgnoreCase(columnName)) { - return false; - } - } - return true; - } - - private void closeConnections() throws KettleException { - try { - for (Entry entry : cons.entrySet()) { - entry.getValue().close(); - } - cons.clear(); - } catch (Exception ex) { - throw new KettleException(ex.getMessage(), ex); - } - } - - /** - * According to the hierarchies,generate the varLengthKeyGenerator - * - * @param keyGenerators - * @param hirches - * @param dimLens - */ - private void updateHierarchyKeyGenerators(Map keyGenerators, - Map hirches, int[] dimLens, String[] dimCols) { - // - String timeHierNameVal = ""; - if (meta.getCarbonTime() == null || "".equals(meta.getCarbonTime())) { - timeHierNameVal = ""; - } else { - String[] hies = meta.getCarbonTime().split(":"); - timeHierNameVal = hies[1]; - } - - // Set> hierSet = hirches.entrySet(); - Iterator> itr = hirches.entrySet().iterator(); - - while (itr.hasNext()) { - Entry hieEntry = itr.next(); - - int[] a = hieEntry.getValue(); - int[] lens = new int[a.length]; - String name = hieEntry.getKey(); - // - if (name.equalsIgnoreCase(timeHierNameVal)) { - for (int i = 0; i < a.length; i++) { //CHECKSTYLE:OFF - lens[i] = dimLens[a[i]]; - } //CHECKSTYLE:ON - } else { - String[] columns = meta.hierColumnMap.get(name); - - if (meta.getComplexTypes().get(columns[0]) != null) { - continue; - } - boolean isNoDictionary = false; - for (int i = 0; i < a.length; i++) { - if (null != meta.noDictionaryCols && isDimensionNoDictionary(meta.noDictionaryCols, - columns[i])) { - isNoDictionary = true; - break; - } - } - //if no dictionary column then do not populate the dim lens - if (isNoDictionary) { - continue; - } - // - for (int i = 0; i < a.length; i++) { - int newIndex = -1; - for (int j = 0; j < dimCols.length; j++) { - // - if (checkDimensionColName(dimCols[j], columns[i])) { - newIndex = j; - break; - } - } //CHECKSTYLE:OFF - lens[i] = dimLens[newIndex]; - } //CHECKSTYLE:ON - } - // - KeyGenerator generator = KeyGeneratorFactory.getKeyGenerator(lens); - keyGenerators.put(name, generator); - - } - - Iterator> complexMap = - meta.getComplexTypes().entrySet().iterator(); - while (complexMap.hasNext()) { - Entry complexDataType = complexMap.next(); - List primitiveTypes = new ArrayList(); - complexDataType.getValue().getAllPrimitiveChildren(primitiveTypes); - for (GenericDataType eachPrimitive : primitiveTypes) { - KeyGenerator generator = KeyGeneratorFactory.getKeyGenerator(new int[] { -1 }); - keyGenerators.put(eachPrimitive.getName(), generator); - } - } - } - - private boolean checkDimensionColName(String dimColName, String hierName) { - String[] tables = meta.getDimTableArray(); - - for (String table : tables) { - String hierWithTableName = table + '_' + hierName; - if (hierWithTableName.equalsIgnoreCase(dimColName)) { - return true; - } - } - - return false; - } - - public boolean init(StepMetaInterface smi, StepDataInterface sdi) { - meta = (CarbonCSVBasedSeqGenMeta) smi; - data = (CarbonCSVBasedSeqGenData) sdi; - return super.init(smi, sdi); - } - - public void dispose(StepMetaInterface smi, StepDataInterface sdi) { - /** - * Fortify Fix: FORWARD_NULL - * Changed to || - * previously there was && but actully in case any one the object being null can through the - * nullpointer exception - * - */ - if (null == smi || null == sdi) { - return; - } - - meta = (CarbonCSVBasedSeqGenMeta) smi; - data = (CarbonCSVBasedSeqGenData) sdi; - CarbonCSVBasedDimSurrogateKeyGen surKeyGen = data.getSurrogateKeyGen(); - - try { - closeConnections(); - if (null != surKeyGen) { - surKeyGen.setHierCache(null); - surKeyGen.setHierCacheReverse(null); - surKeyGen.setTimeDimCache(null); - surKeyGen.setMax(null); - surKeyGen.setTimDimMax(null); - surKeyGen.close(); - } - } catch (Exception e) { - LOGGER.error(e); - } finally { - if (null != surKeyGen) { - clearDictionaryCache(); - surKeyGen.setDictionaryCaches(null); - } - } - nrmlizedHierWriterMap = null; - data.clean(); - super.dispose(smi, sdi); - meta = null; - data = null; - } - - /** - * This method will clear the dictionary access count so that any unused - * column can be removed from the cache - */ - private void clearDictionaryCache() { - Map dictionaryCaches = surrogateKeyGen.getDictionaryCaches(); - List reverseDictionaries = new ArrayList<>(dictionaryCaches.values()); - for (int i = 0; i < reverseDictionaries.size(); i++) { - Dictionary dictionary = reverseDictionaries.get(i); - dictionary.clear(); - } - } - - private String processnoDictionaryDim(int index, String dimensionValue, String dataType, - boolean isStringDataType, ByteBuffer[] out) { - if (!(isStringDataType)) { - if (null == DataTypeUtil - .normalizeIntAndLongValues(dimensionValue, DataTypeUtil.getDataType(dataType))) { - dimensionValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL; - } - } - ByteBuffer buffer = ByteBuffer - .wrap(dimensionValue.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET))); - buffer.rewind(); - out[index] = buffer; - return dimensionValue; - } - - /** - * @param NoDictionaryDims - * @param columnName - * @return true if the dimension is high cardinality. - */ - private boolean isDimensionNoDictionary(String[] NoDictionaryDims, String columnName) { - for (String colName : NoDictionaryDims) { - if (colName - .equalsIgnoreCase(meta.getTableName() + CarbonCommonConstants.UNDERSCORE + columnName)) { - return true; - } - } - return false; - } - - /** - * Observer class for thread execution - * In case of any failure we need stop all the running thread - */ - private class ThreadStatusObserver { - /** - * Below method will be called if any thread fails during execution - * - * @param exception - */ - public void notifyFailed(Throwable exception) throws RuntimeException { - exec.shutdownNow(); - LOGGER.error(exception); - throw new RuntimeException(exception); - } - } - - /** - * This method will fill the carbon measures - * - * @param measures - */ - private void populateCarbonMeasures(String[] measures) { - CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable( - meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName()); - meta.carbonMeasures = new CarbonMeasure[measures.length]; - msrDataType = new DataType[measures.length]; - for (int i = 0; i < measures.length; i++) { - CarbonMeasure carbonMeasure = carbonTable.getMeasureByName(meta.getTableName(), measures[i]); - msrDataType[i] = carbonMeasure.getDataType(); - if (DataType.DECIMAL == carbonMeasure.getDataType()) { - meta.carbonMeasures[i] = carbonMeasure; - } - } - } - - private CarbonDimension[] populateNameToCarbonDimensionMap() { - CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable( - meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName()); - List dimensionsList = carbonTable.getDimensionByTableName(meta.getTableName()); - CarbonDimension[] dimensionOrdinalToDimensionMapping = - new CarbonDimension[meta.getColumnSchemaDetailsWrapper().getColumnSchemaDetailsMap() - .size()]; - List dimListExcludingNoDictionaryColumn = dimensionsList; - if (null != meta.getNoDictionaryDims() && meta.getNoDictionaryDims().length() > 0) { - dimListExcludingNoDictionaryColumn = - new ArrayList<>(dimensionsList.size() - meta.noDictionaryCols.length); - for (CarbonDimension dimension : dimensionsList) { - // Here if dimension.getEncoder() lnly contains Encoding.INVERTED_INDEX, it - // means that NoDicColumn using InvertedIndex, so not put it into dic dims list. - if (!dimension.getEncoder().isEmpty() && !((1 == dimension.getEncoder().size()) && - dimension.getEncoder().contains(Encoding.INVERTED_INDEX))) { - dimListExcludingNoDictionaryColumn.add(dimension); - } - } - } - for (int i = 0; i < dimListExcludingNoDictionaryColumn.size(); i++) { - CarbonDimension dimension = dimListExcludingNoDictionaryColumn.get(meta.memberMapping[i]); - if (dimension.isComplex()) { - populateComplexDimension(dimensionOrdinalToDimensionMapping, dimension); - } else { - dimensionOrdinalToDimensionMapping[meta.memberMapping[i]] = dimension; - } - } - return dimensionOrdinalToDimensionMapping; - } - - private void populateComplexDimension(CarbonDimension[] dimensionOrdinalToDimensionMapping, - CarbonDimension dimension) { - List listOfChildDimensions = dimension.getListOfChildDimensions(); - for (CarbonDimension childDimension : listOfChildDimensions) { - if (childDimension.isComplex()) { - populateComplexDimension(dimensionOrdinalToDimensionMapping, childDimension); - } else { - dimensionOrdinalToDimensionMapping[childDimension.getOrdinal()] = childDimension; - } - } - } - -} - diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java deleted file mode 100644 index eb0f52b3a14..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java +++ /dev/null @@ -1,339 +0,0 @@ -/* - * 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.surrogatekeysgenerator.csvbased; - -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -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.CarbonCommonConstants; -import org.apache.carbondata.core.keygenerator.KeyGenException; -import org.apache.carbondata.core.keygenerator.KeyGenerator; -import org.apache.carbondata.core.metadata.CarbonMetadata; -import org.apache.carbondata.core.metadata.CarbonTableIdentifier; -import org.apache.carbondata.core.metadata.ColumnIdentifier; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory; -import org.apache.carbondata.core.writer.ByteArrayHolder; -import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV; -import org.apache.carbondata.processing.datatypes.GenericDataType; -import org.apache.carbondata.processing.mdkeygen.file.FileData; -import org.apache.carbondata.processing.mdkeygen.file.FileManager; -import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper; -import org.apache.carbondata.processing.schema.metadata.ColumnsInfo; -import org.apache.carbondata.processing.util.CarbonDataProcessorUtil; - -import org.pentaho.di.core.exception.KettleException; - -public class FileStoreSurrogateKeyGenForCSV extends CarbonCSVBasedDimSurrogateKeyGen { - - /** - * hierValueWriter - */ - private Map hierValueWriter; - - /** - * keyGenerator - */ - private Map keyGenerator; - - /** - * LOAD_FOLDER - */ - private String loadFolderName; - - /** - * primaryKeyStringArray - */ - private String[] primaryKeyStringArray; - /** - * partitionID - */ - private String partitionID; - /** - * load Id - */ - private String segmentId; - /** - * task id, each spark task has a unique id - */ - private String taskNo; - - /** - * @param columnsInfo - * @throws IOException - */ - public FileStoreSurrogateKeyGenForCSV(ColumnsInfo columnsInfo, String partitionID, - String segmentId, String taskNo) throws IOException { - super(columnsInfo); - populatePrimaryKeyarray(dimInsertFileNames, columnsInfo.getPrimaryKeyMap()); - this.partitionID = partitionID; - this.segmentId = segmentId; - this.taskNo = taskNo; - keyGenerator = new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - setStoreFolderWithLoadNumber( - checkAndCreateLoadFolderNumber(columnsInfo.getDatabaseName(), - columnsInfo.getTableName())); - fileManager = new FileManager(); - fileManager.setName(loadFolderName + CarbonCommonConstants.FILE_INPROGRESS_STATUS); - - hierValueWriter = new HashMap( - CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - - for (Entry entry : hierInsertFileNames.entrySet()) { - String hierFileName = entry.getValue().trim(); - hierValueWriter.put(entry.getKey(), - new HierarchyValueWriterForCSV(hierFileName, getStoreFolderWithLoadNumber())); - Map keyGenerators = columnsInfo.getKeyGenerators(); - keyGenerator.put(entry.getKey(), keyGenerators.get(entry.getKey())); - FileData fileData = new FileData(hierFileName, getStoreFolderWithLoadNumber()); - fileData.setHierarchyValueWriter(hierValueWriter.get(entry.getKey())); - fileManager.add(fileData); - } - populateCache(); - //Update the primary key surroagate key map - updatePrimaryKeyMaxSurrogateMap(); - } - - private void populatePrimaryKeyarray(String[] dimInsertFileNames, Map map) { - List primaryKeyList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - for (String columnName : dimInsertFileNames) { - if (null != map.get(columnName)) { - map.put(columnName, false); - } - } - Set> entrySet = map.entrySet(); - for (Entry entry : entrySet) { - if (entry.getValue()) { - primaryKeyList.add(entry.getKey().trim()); - } - } - primaryKeyStringArray = primaryKeyList.toArray(new String[primaryKeyList.size()]); - } - - /** - * update the - */ - private void updatePrimaryKeyMaxSurrogateMap() { - Map primaryKeyMap = columnsInfo.getPrimaryKeyMap(); - for (Entry entry : primaryKeyMap.entrySet()) { - if (!primaryKeyMap.get(entry.getKey())) { - int repeatedPrimaryFromLevels = - getRepeatedPrimaryFromLevels(dimInsertFileNames, entry.getKey()); - - if (null == primaryKeysMaxSurroagetMap) { - primaryKeysMaxSurroagetMap = - new HashMap(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - } - primaryKeysMaxSurroagetMap.put(entry.getKey(), max[repeatedPrimaryFromLevels]); - } - } - } - - private int getRepeatedPrimaryFromLevels(String[] columnNames, String primaryKey) { - for (int j = 0; j < columnNames.length; j++) { - if (primaryKey.equals(columnNames[j])) { - return j; - } - } - return -1; - } - - private String checkAndCreateLoadFolderNumber(String databaseName, - String tableName) throws IOException { - String carbonDataDirectoryPath = CarbonDataProcessorUtil - .getLocalDataFolderLocation(databaseName, tableName, taskNo, partitionID, segmentId + "", - false); - boolean isDirCreated = new File(carbonDataDirectoryPath).mkdirs(); - if (!isDirCreated) { - throw new IOException("Unable to create data load directory" + carbonDataDirectoryPath); - } - return carbonDataDirectoryPath; - } - - /** - * This method will update the maxkey information. - * @param tabColumnName - * @param maxKey max cardinality of a column - */ - private void updateMaxKeyInfo(String tabColumnName, int maxKey) { - checkAndUpdateMap(maxKey, tabColumnName); - } - - /** - * This method will generate cache for all the global dictionaries during data loading. - */ - private void populateCache() throws IOException { - String carbonStorePath = - CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS); - String[] dimColumnNames = columnsInfo.getDimColNames(); - String[] dimColumnIds = columnsInfo.getDimensionColumnIds(); - String databaseName = columnsInfo.getDatabaseName(); - String tableName = columnsInfo.getTableName(); - CarbonTable carbonTable = CarbonMetadata.getInstance() - .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName); - CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier(); - CacheProvider cacheProvider = CacheProvider.getInstance(); - Cache reverseDictionaryCache = - cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath); - List dictionaryKeys = new ArrayList<>(dimColumnNames.length); - List dictionaryColumnUniqueIdentifiers = - new ArrayList<>(dimColumnNames.length); - ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper = - columnsInfo.getColumnSchemaDetailsWrapper(); - // update the member cache for dimension - for (int i = 0; i < dimColumnNames.length; i++) { - String dimColName = dimColumnNames[i].substring(tableName.length() + 1); - ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimColumnIds[i]); - if (details.isDirectDictionary()) { - continue; - } - GenericDataType complexType = columnsInfo.getComplexTypesMap().get(dimColName); - if (complexType != null) { - List primitiveChild = new ArrayList(); - complexType.getAllPrimitiveChildren(primitiveChild); - for (GenericDataType eachPrimitive : primitiveChild) { - details = columnSchemaDetailsWrapper.get(eachPrimitive.getColumnId()); - if (details.isDirectDictionary()) { - continue; - } - ColumnIdentifier columnIdentifier = new ColumnIdentifier(eachPrimitive.getColumnId(), - columnsInfo.getColumnProperties(eachPrimitive.getName()), details.getColumnType()); - String dimColumnName = - tableName + CarbonCommonConstants.UNDERSCORE + eachPrimitive.getName(); - DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier = - new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier); - dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier); - dictionaryKeys.add(dimColumnName); - } - } else { - ColumnIdentifier columnIdentifier = - new ColumnIdentifier(dimColumnIds[i], columnsInfo.getColumnProperties(dimColName), - details.getColumnType()); - DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier = - new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier); - dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier); - dictionaryKeys.add(dimColumnNames[i]); - } - } - initDictionaryCacheInfo(dictionaryKeys, dictionaryColumnUniqueIdentifiers, - reverseDictionaryCache); - } - - /** - * This method will initial the needed information for a dictionary of one column. - * - * @param dictionaryKeys - * @param dictionaryColumnUniqueIdentifiers - * @param reverseDictionaryCache - * @throws KettleException - */ - private void initDictionaryCacheInfo(List dictionaryKeys, - List dictionaryColumnUniqueIdentifiers, - Cache reverseDictionaryCache) throws IOException { - long lruCacheStartTime = System.currentTimeMillis(); - List reverseDictionaries = reverseDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers); - for (int i = 0; i < reverseDictionaries.size(); i++) { - Dictionary reverseDictionary = (Dictionary) reverseDictionaries.get(i); - getDictionaryCaches().put(dictionaryKeys.get(i), reverseDictionary); - updateMaxKeyInfo(dictionaryKeys.get(i), reverseDictionary.getDictionaryChunks().getSize()); - } - CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordLruCacheLoadTime( - (System.currentTimeMillis() - lruCacheStartTime) / 1000.0); - } - - @Override protected int getSurrogateFromStore(String value, int index, Object[] properties) - throws KettleException { - max[index]++; - int key = max[index]; - return key; - } - - @Override - protected int updateSurrogateToStore(String tuple, String columnName, int index, int key, - Object[] properties) throws KettleException { - Map cache = getTimeDimCache().get(columnName); - if (cache == null) { - return key; - } - return key; - } - - private void checkAndUpdateMap(int maxKey, String dimInsertFileNames) { - String[] dimsFiles2 = getDimsFiles(); - for (int i = 0; i < dimsFiles2.length; i++) { - if (dimInsertFileNames.equalsIgnoreCase(dimsFiles2[i])) { - if (max[i] < maxKey) { - max[i] = maxKey; - break; - } - } - } - - } - - @Override public boolean isCacheFilled(String[] columns) { - for (String column : columns) { - Dictionary dicCache = getDictionaryCaches().get(column); - if (null == dicCache) { - return true; - } - } - return false; - } - - public IFileManagerComposite getFileManager() { - return fileManager; - } - - @Override protected byte[] getNormalizedHierFromStore(int[] val, String hier, int primaryKey, - HierarchyValueWriterForCSV hierWriter) throws KettleException { - byte[] bytes; - try { - bytes = columnsInfo.getKeyGenerators().get(hier).generateKey(val); - hierWriter.getByteArrayList().add(new ByteArrayHolder(bytes, primaryKey)); - } catch (KeyGenException e) { - throw new KettleException(e); - } - return bytes; - } - - @Override public int getSurrogateForMeasure(String tuple, String columnName) - throws KettleException { - Integer measureSurrogate = null; - Map dictionaryCaches = getDictionaryCaches(); - Dictionary dicCache = dictionaryCaches.get(columnName); - measureSurrogate = dicCache.getSurrogateKey(tuple); - return measureSurrogate; - } - -} diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties deleted file mode 100644 index d5b97fa2572..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties +++ /dev/null @@ -1,61 +0,0 @@ -# -# 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. -# -CarbonDialog.Shell.Title=Carbon Surrogate Generator -CarbonDialog.Group.Label=Carbon Settings -CarbonDialog.dim.Label=Dimensions -CarbonDialog.dim.Tooltip=Dimension index separated by comma(,) -CarbonDialog.msr.Label=Measures -CarbonDialog.msr.Tooltip=Measures index separated by comma(,) -CarbonDialog.hier.Label=Hierarichies -CarbonDialog.hier.Tooltip=Hierarichies seperated by &. - -CarbonDialog.time.Label=Time Hierarichy -CarbonDialog.time.Tooltip=Time Hierarichy seperated by , - -CarbonDialog.loc.Label=Location -CarbonDialog.loc.Tooltip=Location to keep hierarichy files - -CarbonDialog.con.Label=Connection -CarbonDialog.con.Tooltip=Jdbc url - -CarbonDialog.Schema.Label=Schema - -CarbonDialog.BatchSize.Label=Carbon Batch Size -CarbonDialog.BatchSize.Tooltip=Carbon Batch Size - -CarbonDialog.CarbonProperties.Label=Carbon Properties -CarbonDialog.CarbonProperties.Tooltip=Carbon Properties - -CarbonDialog.MetaHier.Label=Carbon Metadata Hierarichies -CarbonDialog.MetaHier.Tooltip=Carbon Metadata Hierarichies - -CarbonDialog.IsInitialLoad=Initial Load - -CarbonDialog.MetadataFolderLocation.Label= Carbon Metadata Files Location -CarbonDialog.MetadataFolderLocation.Tooltip= Carbon Metadata Files Location - -CarbonDialog.DoMapping.UnableToFindInput=Cannot find previous fields -CarbonStep.Exception.UnexpectedErrorInReadingStepInfo=Error reading step info -CarbonStep.Exception.UnableToSaveStepInfoToRepository=Error saving step info - -CarbonStep.Error.UnableFindField=Unable to find key field -CarbonStep.Check.StepIsReceivingInfoFromOtherSteps=Step is receiving input from other steps -CarbonStep.Check.NoInputReceivedFromOtherSteps=Step is not receiving any input from other steps -CarbonStep.Check.MissingFieldsNotFoundInInput=Required fields not found in input -CarbonStep.Check.AllFieldsFoundInInput=All required fields found -CarbonStep.Check.CouldNotReadFromPreviousSteps=Could not read from previous steps diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java index bde8215a93f..27662a4b623 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java +++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java @@ -55,12 +55,6 @@ import org.apache.carbondata.processing.newflow.DataField; import org.apache.commons.lang3.ArrayUtils; -import org.pentaho.di.core.CheckResult; -import org.pentaho.di.core.CheckResultInterface; -import org.pentaho.di.core.Const; -import org.pentaho.di.core.row.RowMetaInterface; -import org.pentaho.di.i18n.BaseMessages; -import org.pentaho.di.trans.step.StepMeta; public final class CarbonDataProcessorUtil { private static final LogService LOGGER = @@ -149,9 +143,7 @@ public static void renameBadRecordsFromInProgressToNormal(String storeLocation) String badRecordsInProgressFileName = null; String changedFileName = null; - // CHECKSTYLE:OFF for (CarbonFile badFiles : listFiles) { - // CHECKSTYLE:ON badRecordsInProgressFileName = badFiles.getName(); changedFileName = badLogStoreLocation + File.separator + badRecordsInProgressFileName @@ -164,51 +156,6 @@ public static void renameBadRecordsFromInProgressToNormal(String storeLocation) LOGGER.error("Unable to delete File : " + badFiles.getName()); } } - } // CHECKSTYLE:ON - } - - public static void checkResult(List remarks, StepMeta stepMeta, - String[] input) { - CheckResult cr; - - // See if we have input streams leading to this step! - if (input.length > 0) { - cr = new CheckResult(CheckResult.TYPE_RESULT_OK, "Step is receiving info from other steps.", - stepMeta); - remarks.add(cr); - } else { - cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, "No input received from other steps!", - stepMeta); - remarks.add(cr); - } - } - - public static void check(Class pkg, List remarks, StepMeta stepMeta, - RowMetaInterface prev, String[] input) { - CheckResult cr; - - // See if we have input streams leading to this step! - if (input.length > 0) { - cr = new CheckResult(CheckResult.TYPE_RESULT_OK, - BaseMessages.getString(pkg, "CarbonStep.Check.StepIsReceivingInfoFromOtherSteps"), - stepMeta); - remarks.add(cr); - } else { - cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, - BaseMessages.getString(pkg, "CarbonStep.Check.NoInputReceivedFromOtherSteps"), stepMeta); - remarks.add(cr); - } - - // also check that each expected key fields are acually coming - if (prev != null && prev.size() > 0) { - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK, - BaseMessages.getString(pkg, "CarbonStep.Check.AllFieldsFoundInInput"), stepMeta); - remarks.add(cr); - } else { - String errorMessage = - BaseMessages.getString(pkg, "CarbonStep.Check.CouldNotReadFromPreviousSteps") + Const.CR; - cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR, errorMessage, stepMeta); - remarks.add(cr); } } diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java deleted file mode 100644 index bba40e9ef5c..00000000000 --- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java +++ /dev/null @@ -1,1076 +0,0 @@ -/* - * 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.util; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory; -import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.encoder.Encoding; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; -import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; -import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema.DimensionRelation; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails; -import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper; - -public final class CarbonSchemaParser { - /** - * - */ - public static final String QUOTES = "\""; - - private CarbonSchemaParser() { - - } - - /** - * This method Return the dimension queries based on quotest required or not. - * - * @param dimensions - * @return - */ - public static String getDimensionSQLQueries(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema, boolean isQuotesRequired, String quote) { - if (isQuotesRequired) { - return getDimensionSQLQueriesWithQuotes(dimensions, carbonDataLoadSchema, quote); - } else { - return getDimensionSQLQueries(dimensions, carbonDataLoadSchema); - } - } - - public static String getDenormColNames(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - // - List foreignKeys = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - Set allRelationCols = new HashSet(); - - for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) { - foreignKeys.add(dimensionRelation.getRelation().getFactForeignKeyColumn()); - allRelationCols.addAll(dimensionRelation.getColumns()); - } - - StringBuilder columns = new StringBuilder(); - - for (CarbonDimension dim : dimensions) { - if (foreignKeys.contains(dim.getColName()) && !allRelationCols.contains(dim.getColName())) { - columns.append(dim.getColName()); - columns.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - } - - String columnstr = columns.toString(); - if (columnstr.length() > 0 && columnstr.endsWith(CarbonCommonConstants.HASH_SPC_CHARACTER)) { - columnstr = columnstr - .substring(0, columnstr.length() - CarbonCommonConstants.HASH_SPC_CHARACTER.length()); - } - - return columnstr; - } - - private static String getDimensionSQLQueries(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - // - List queryList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - for (CarbonDimension dim : dimensions) { - - String tableName = extractDimensionTableName(dim.getColName(), carbonDataLoadSchema); - StringBuilder query; - String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName(); - if (factTableName.equals(tableName)) { - continue; - } - String dimName = dim.getColName(); - query = - new StringBuilder(dimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER); - - String primaryKey = null; - for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) { - for (String field : dimensionRelation.getColumns()) { - if (dimName.equals(field)) { - primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn(); - break; - } - } - if (null != primaryKey) { - break; - } - } - query.append("SELECT "); - query.append(primaryKey + ','); - query.append(dimName); - query.append(" FROM " + tableName); - queryList.add(query.toString()); - } - StringBuilder finalQuryString = new StringBuilder(); - - for (int i = 0; i < queryList.size() - 1; i++) { - finalQuryString.append(queryList.get(i)); - finalQuryString.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - if (queryList.size() > 0) { - finalQuryString.append(queryList.get(queryList.size() - 1)); - } - return finalQuryString.toString(); - } - - /** - * @param table - * @param dimensions - * @return - */ - - private static String getDimensionSQLQueriesWithQuotes(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema, String quotes) { - // - List queryList = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - // Property[] properties = null; - for (CarbonDimension dim : dimensions) { - - String tableName = extractDimensionTableName(dim.getColName(), carbonDataLoadSchema); - StringBuilder query; - String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName(); - if (factTableName.equals(tableName)) { - continue; - } - String dimName = dim.getColName(); - query = - new StringBuilder(dimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER); - - String primaryKey = null; - for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) { - for (String field : dimensionRelation.getColumns()) { - if (dimName.equals(field)) { - primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn(); - break; - } - } - if (null != primaryKey) { - break; - } - } - query.append("SELECT "); - query.append(quotes + primaryKey + quotes + ','); - query.append(quotes + dimName + quotes); - query.append(" FROM " + quotes + tableName + quotes); - queryList.add(query.toString()); - } - StringBuilder finalQuryString = new StringBuilder(); - - for (int i = 0; i < queryList.size() - 1; i++) { - finalQuryString.append(queryList.get(i)); - finalQuryString.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - if (queryList.size() > 0) { - finalQuryString.append(queryList.get(queryList.size() - 1)); - } - return finalQuryString.toString(); - } - - /** - * @param dimensions - * @param measures - * @param factTableName - * @param isQuotesRequired - * @param schemaInfo - * @return - */ - public static String getTableInputSQLQuery(List dimensions, - List measures, String factTableName, boolean isQuotesRequired, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder query = new StringBuilder("SELECT "); - - getQueryForDimension(dimensions, query, factTableName, isQuotesRequired, carbonDataLoadSchema); - - if (!"select".equalsIgnoreCase(query.toString().trim())) { - query.append(","); - } - Set uniqueMsrCols = new HashSet(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - for (int i = 0; i < measures.size(); i++) { - - uniqueMsrCols.add(measures.get(i).getColName()); - } - String[] uniqueMeasure = uniqueMsrCols.toArray(new String[uniqueMsrCols.size()]); - for (int j = 0; j < uniqueMeasure.length; j++) { - query.append(System.getProperty("line.separator")); - if (isQuotesRequired) { - query.append(QUOTES + uniqueMeasure[j] + QUOTES); - } else { - query.append(uniqueMeasure[j]); - } - - if (j != uniqueMeasure.length - 1) { - - query.append(","); - } - } - query.append(System.getProperty("line.separator")); - - if (isQuotesRequired) { - query.append(" FROM " + QUOTES + factTableName + QUOTES + ' '); - } else { - query.append(" FROM " + factTableName + ' '); - } - - return query.toString(); - } - - private static void getQueryForDimension(List dimensions, StringBuilder query, - String factTableName, boolean isQuotesRequired, CarbonDataLoadSchema carbonDataLoadSchema) { - int counter = 0; - for (CarbonDimension cDim : dimensions) { - - String foreignKey = null; - for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) { - for (String field : dimensionRelation.getColumns()) { - if (cDim.getColName().equals(field)) { - foreignKey = dimensionRelation.getRelation().getFactForeignKeyColumn(); - } - } - } - if (foreignKey != null) { - query.append(System.getProperty("line.separator")); - if (counter != 0) { - query.append(','); - } - - if (isQuotesRequired) { - query.append(QUOTES + foreignKey + QUOTES); - } else { - query.append(foreignKey); - } - continue; - } else { - query.append(System.getProperty("line.separator")); - if (counter != 0) { - query.append(','); - } - - if (isQuotesRequired) { - query.append(QUOTES + factTableName + QUOTES + '.' + QUOTES + cDim.getColName() + QUOTES); - } else { - query.append(factTableName + '.' + cDim.getColName()); - } - } - counter++; - } - } - - /** - * Get dimension string from a array of TableDimension,which can be shared - * TableDimension within schema or in a table. - * - * @param table - * @param dimensions - * @return - */ - public static int getDimensionString(List dimensions, StringBuilder dimString, - int counter, CarbonDataLoadSchema carbonDataLoadSchema) { - for (CarbonDimension cDimension : dimensions) { - if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - dimString.append( - tableName + '_' + cDimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER - + counter + CarbonCommonConstants.COLON_SPC_CHARACTER + -1 - + CarbonCommonConstants.COLON_SPC_CHARACTER + 'Y' - + CarbonCommonConstants.COMA_SPC_CHARACTER); - counter++; - } - return counter; - } - - /** - * Return mapping of Column name to cardinality - */ - - public static Map getCardinalities(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - Map cardinalities = new LinkedHashMap(); - for (CarbonDimension cDimension : dimensions) { - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - cardinalities.put(tableName + '_' + cDimension.getColName(), -1 + ""); - } - return cardinalities; - } - - /** - * Get measure string from a array of Measure - * - * @param measures - * @return - */ - public static String getMeasureString(List measures, int counter) { - StringBuilder measureString = new StringBuilder(); - int i = measures.size(); - for (CarbonMeasure measure : measures) { - - measureString - .append(measure.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + counter); - counter++; - if (i > 1) { - measureString.append(CarbonCommonConstants.COMA_SPC_CHARACTER); - } - i--; - - } - return measureString.toString(); - } - - /** - * Get measure string from a array of Measure - * - * @param measures - * @return - */ - public static String[] getMeasures(List measures) { - String[] measuresStringArray = new String[measures.size()]; - - for (int i = 0; i < measuresStringArray.length; i++) { - measuresStringArray[i] = measures.get(i).getColName(); - } - return measuresStringArray; - } - - //TODO SIMIAN - - /** - * Get hierarchy string from dimensions - * - * @param dimensions - * @return - */ - public static String getHierarchyString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder hierString = new StringBuilder(); - String hierStr = ""; - - for (CarbonDimension cDimension : dimensions) { - if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String cDimName = cDimension.getColName(); - hierStr = 0 + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER; - hierStr = cDimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER + hierStr; - hierString.append(hierStr); - } - - hierStr = hierString.toString(); - if (hierStr.length() > 0 && hierStr.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - hierStr = hierStr - .substring(0, hierStr.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return hierStr; - } - - /** - * this method will return table columns - * - * @param dimensions - * @param carbonDataLoadSchema - * @return - */ - public static String[] getTableDimensions(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - List list = new ArrayList(CarbonCommonConstants.CONSTANT_SIZE_TEN); - for (CarbonDimension cDimension : dimensions) { - // Ignoring the dimensions which are high cardinality dimension - if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - list.add(extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema) + "_" - + cDimension.getColName()); - } - String[] fields = new String[list.size()]; - fields = list.toArray(fields); - return fields; - } - - /** - * This method will extract dimension table name, - * By default, fact table name will be returned. - * - * @param dimensionColName - * @param carbonDataLoadSchema - * @return - */ - private static String extractDimensionTableName(String dimensionColName, - CarbonDataLoadSchema carbonDataLoadSchema) { - List dimensionRelationList = carbonDataLoadSchema.getDimensionRelationList(); - - for (DimensionRelation dimensionRelation : dimensionRelationList) { - for (String field : dimensionRelation.getColumns()) { - if (dimensionColName.equals(field)) { - return dimensionRelation.getTableName(); - } - } - } - return carbonDataLoadSchema.getCarbonTable().getFactTableName(); - } - - /** - * It will return all column groups in below format - * 0,1~2~3,4,5,6~7~8,9 - * groups are - * ,-> all ordinal with different group id - * ~-> all ordinal with same group id - * - * @param dimensions - * @return - */ - public static String getColumnGroups(List dimensions) { - StringBuilder columnGroups = new StringBuilder(); - for (int i = 0; i < dimensions.size(); i++) { - CarbonDimension dimension = dimensions.get(i); - //assuming complex dimensions will always be atlast - if (null != dimension.getListOfChildDimensions() && - dimension.getListOfChildDimensions().size() > 0) { - break; - } - if (!dimension.hasEncoding(Encoding.DICTIONARY)) { - continue; - } - columnGroups.append(dimension.getOrdinal()); - if (i < dimensions.size() - 1) { - int currGroupOrdinal = dimension.columnGroupId(); - int nextGroupOrdinal = dimensions.get(i + 1).columnGroupId(); - if (currGroupOrdinal == nextGroupOrdinal && currGroupOrdinal != -1) { - columnGroups.append("~"); - } else { - columnGroups.append(","); - } - } - - } - return columnGroups.toString(); - } - - /** - * getHeirAndCardinalityString - * - * @param dimensions - * @param schema - * @return String - */ - public static String getHeirAndCardinalityString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder builder = new StringBuilder(); - String heirName = null; - for (CarbonDimension cDimension : dimensions) { - heirName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String dimName = cDimension.getColName(); - builder.append(dimName + '_' + heirName + ".hierarchy"); - builder.append(CarbonCommonConstants.COLON_SPC_CHARACTER); - builder.append(-1); - builder.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - return builder.toString(); - } - - /** - * @param dimensions - * @return - */ - public static String getMetaHeirString(List dimensions, CarbonTable schema) { - StringBuilder propString = new StringBuilder(); - String tableName = schema.getFactTableName(); - for (CarbonDimension cDimension : dimensions) { - propString.append(tableName + "_" + cDimension.getColName()); - propString.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - - } - // Delete the last special character - String prop = propString.toString(); - if (prop.endsWith(CarbonCommonConstants.HASH_SPC_CHARACTER)) { - prop = prop.substring(0, prop.length() - CarbonCommonConstants.HASH_SPC_CHARACTER.length()); - } - return prop; - } - - public static String getTableNameString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder stringBuffer = new StringBuilder(); - - for (CarbonDimension cDimension : dimensions) { - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - - stringBuffer.append(cDimension.getColName() + '_' + cDimension.getColName()); - stringBuffer.append(CarbonCommonConstants.COLON_SPC_CHARACTER); - stringBuffer.append(tableName); - stringBuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - // Delete the last & character - String string = stringBuffer.toString(); - if (string.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - string = string - .substring(0, string.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return string; - } - - /** - * This method will concatenate all the column ids for a given list of dimensions - * - * @param dimensions - * @return - */ - public static String getColumnIdString(List dimensions) { - StringBuilder stringBuffer = new StringBuilder(); - for (CarbonDimension cDimension : dimensions) { - if (!cDimension.hasEncoding(Encoding.DICTIONARY)) { - continue; - } - stringBuffer.append(cDimension.getColumnId()); - stringBuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - // Delete the last & character - String string = stringBuffer.toString(); - if (string.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - string = string - .substring(0, string.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return string; - } - - /** - * @param dimensions - * @param schema - * @return - */ - public static String getMdkeySizeForFact(List dimensions) { - int[] dims = new int[dimensions.size()]; - for (int i = 0; i < dims.length; i++) { - dims[i] = -1; - } - return KeyGeneratorFactory.getKeyGenerator(dims).getKeySizeInBytes() + ""; - } - - /** - * @param dimensions - * @param schema - * @return - */ - public static String getHeirAndKeySizeMapForFact(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder stringBuffer = new StringBuilder(); - String heirName = null; - int[] dims = null; - int keySizeInBytes = 0; - for (CarbonDimension cDimension : dimensions) { - String dimName = cDimension.getColName(); - heirName = extractDimensionTableName(dimName, carbonDataLoadSchema); - dims = new int[] { -1 }; - keySizeInBytes = KeyGeneratorFactory.getKeyGenerator(dims).getKeySizeInBytes(); - stringBuffer.append(dimName + '_' + heirName + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION - + CarbonCommonConstants.COLON_SPC_CHARACTER + keySizeInBytes - + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - return stringBuffer.toString(); - } - - /** - * @param dimensions - * @return - */ - public static String getHierarchyStringWithColumnNames(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - - StringBuilder hierString = new StringBuilder(); - String hierStr = ""; - - for (CarbonDimension cDimension : dimensions) { - if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String cDimName = cDimension.getColName(); - hierStr = cDimName + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER; - hierStr = cDimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER + hierStr; - hierString.append(hierStr); - } - - hierStr = hierString.toString(); - if (hierStr.length() > 0 && hierStr.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - hierStr = hierStr - .substring(0, hierStr.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return hierStr; - - } - - /** - * Return foreign key array - * - * @param dimensions - * @return - */ - public static String[] getForeignKeyForTables(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - Set foreignKey = new LinkedHashSet(); - for (CarbonDimension cDimension : dimensions) { - - List dimensionRelationList = - carbonDataLoadSchema.getDimensionRelationList(); - - for (DimensionRelation dimensionRelation : dimensionRelationList) { - for (String field : dimensionRelation.getColumns()) { - if (cDimension.getColName().equals(field)) { - foreignKey.add(dimensionRelation.getRelation().getFactForeignKeyColumn()); - } - } - } - - } - return foreignKey.toArray(new String[foreignKey.size()]); - - } - - /** - * Return foreign key and respective hierarchy String. - * - * @param dimensions - * @return - */ - public static String getForeignKeyHierarchyString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema, String factTable) { - StringBuilder foreignKeyHierarchyString = new StringBuilder(); - String columns = ""; - - for (CarbonDimension cDimension : dimensions) { - String dimTableName = - extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String dimName = cDimension.getColName(); - - if (dimTableName.equals(factTable)) { - continue; - } - - String foreignKey = null; - for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) { - for (String field : dimensionRelation.getColumns()) { - if (dimName.equals(field)) { - foreignKey = dimensionRelation.getRelation().getFactForeignKeyColumn(); - break; - } - } - - foreignKeyHierarchyString.append(foreignKey); - foreignKeyHierarchyString.append(CarbonCommonConstants.COLON_SPC_CHARACTER); - foreignKeyHierarchyString.append(dimName + '_' + dimTableName); - foreignKeyHierarchyString.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - } - columns = foreignKeyHierarchyString.toString(); - if (columns.length() > 0 && columns.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - columns = columns - .substring(0, columns.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return columns; - - } - - /** - * Return foreign key and respective hierarchy String. - * - * @param dimensions - * @param factTableName - * @return - */ - public static String getForeignKeyAndPrimaryKeyMapString( - List dimensionRelationList) { - StringBuilder foreignKeyHierarchyString = new StringBuilder(); - String columns = ""; - - for (DimensionRelation dimensionRelation : dimensionRelationList) { - foreignKeyHierarchyString.append(dimensionRelation.getRelation().getFactForeignKeyColumn()); - foreignKeyHierarchyString.append(CarbonCommonConstants.COLON_SPC_CHARACTER); - foreignKeyHierarchyString.append( - dimensionRelation.getTableName() + '_' + dimensionRelation.getRelation() - .getDimensionPrimaryKeyColumn()); - foreignKeyHierarchyString.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - columns = foreignKeyHierarchyString.toString(); - if (columns.length() > 0 && columns.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - columns = columns - .substring(0, columns.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return columns; - - } - - /** - * Return foreign key array - * - * @param dimensions - * @return - */ - public static String getPrimaryKeyString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder primaryKeyStringbuffer = new StringBuilder(); - for (CarbonDimension cDimension : dimensions) { - String dimTableName = - extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String dimName = cDimension.getColName(); - - String primaryKey = null; - if (dimTableName.equals(carbonDataLoadSchema.getCarbonTable().getFactTableName())) { - dimTableName = dimName; - } else { - for (DimensionRelation dimensionRelation : carbonDataLoadSchema - .getDimensionRelationList()) { - for (String field : dimensionRelation.getColumns()) { - if (field.equals(dimName)) { - primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn(); - break; - } - } - } - } - - primaryKeyStringbuffer.append(dimTableName + '_' + primaryKey); - primaryKeyStringbuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - - } - - String primaryKeyString = primaryKeyStringbuffer.toString(); - - if (primaryKeyString.length() > 0 && primaryKeyString - .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - primaryKeyString = primaryKeyString.substring(0, - primaryKeyString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - - return primaryKeyString; - } - - /** - * Get Measure Name String - * - * @param table - * @return - */ - public static String getMeasuresNamesString(List measures) { - StringBuilder measureNames = new StringBuilder(); - - for (int i = 0; i < measures.size(); i++) { - measureNames.append(measures.get(i).getColName()); - measureNames.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - - String measureNameString = measureNames.toString(); - - if (measureNameString.length() > 0 && measureNameString - .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - measureNameString = measureNameString.substring(0, - measureNameString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - - return measureNameString; - } - - /** - * Get Measure Name String - * - * @param table - * @return - */ - public static String getMeasuresUniqueColumnNamesString(List measures) { - StringBuilder measureNames = new StringBuilder(); - Set set = new HashSet(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - for (int i = 0; i < measures.size(); i++) { - if (!set.contains(measures.get(i).getColName())) { - set.add(measures.get(i).getColName()); - measureNames.append(measures.get(i).getColName()); - measureNames.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - } - String measureNameString = measureNames.toString(); - if (measureNameString.length() > 0 && measureNameString - .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - measureNameString = measureNameString.substring(0, - measureNameString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - return measureNameString; - } - - /** - * Get Measure Aggregator array - * - * @param table - * @return - */ - public static String[] getMeasuresAggragatorArray(List measures) { - String[] msrAggregators = new String[measures.size()]; - - for (int i = 0; i < msrAggregators.length; i++) { - msrAggregators[i] = "sum"; - } - - return msrAggregators; - } - - /** - * @param schemaInfo - * @param table - * @return - */ - public static String getActualDimensions(List dimensions) { - StringBuilder actualDim = new StringBuilder(); - for (CarbonDimension cDimension : dimensions) { - if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - actualDim.append(cDimension.getColName()); - actualDim.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - - String actualDimString = actualDim.toString(); - - if (actualDimString.length() > 0 && actualDimString - .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - actualDimString = actualDimString.substring(0, - actualDimString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - - return actualDimString; - } - - /**Method will prepare column name and its data type string inorder - * to pass to the ETL steps. - * @param schemaInfo - * @param table - * @return - */ - public static String getDimensionsDataTypes(List dimensions) { - StringBuilder dimDataTypeBuilder = new StringBuilder(); - for (CarbonDimension cDimension : dimensions) { - dimDataTypeBuilder.append(cDimension.getColName()); - dimDataTypeBuilder.append(CarbonCommonConstants.COMA_SPC_CHARACTER); - dimDataTypeBuilder.append(cDimension.getDataType().toString()); - dimDataTypeBuilder.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - String dimDataType = dimDataTypeBuilder.toString(); - return dimDataType; - } - - public static String getMeasuresDataType(List measures) { - StringBuilder measureDataTypeString = new StringBuilder(); - - for (CarbonMeasure measure : measures) { - measureDataTypeString.append(measure.getDataType()) - .append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER); - } - - String measureTypeString = measureDataTypeString.toString(); - - if (measureTypeString.length() > 0 && measureTypeString - .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) { - measureTypeString = measureTypeString.substring(0, - measureTypeString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length()); - } - - return measureTypeString; - - } - - /** - * Below method will be used to get the level and its data type string - * - * @param dimensions - * @param schema - * @param table - * @return String - */ - public static String getLevelAndDataTypeMapString(List dimensions, - CarbonDataLoadSchema carbonDataLoadSchema) { - StringBuilder dimString = new StringBuilder(); - for (CarbonDimension cDimension : dimensions) { - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - String levelName = tableName + '_' + cDimension.getColName(); - dimString.append(levelName + CarbonCommonConstants.LEVEL_FILE_EXTENSION - + CarbonCommonConstants.COLON_SPC_CHARACTER + cDimension.getDataType() - + CarbonCommonConstants.HASH_SPC_CHARACTER); - } - return dimString.toString(); - } - - /** - * Below method will be used to get the complex dimension string - * - * @param dimensions - * @param schema - * @param table - * @return String - */ - public static String getComplexTypeString(List dimensions) { - StringBuilder dimString = new StringBuilder(); - for (int i = 0; i < dimensions.size(); i++) { - CarbonDimension dimension = dimensions.get(i); - if (dimension.getDataType().equals(DataType.ARRAY) || dimension.getDataType() - .equals(DataType.STRUCT)) { - addAllComplexTypeChildren(dimension, dimString, ""); - dimString.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER); - } - } - return dimString.toString(); - } - - /** - * This method will return all the child dimensions under complex dimension - * - * @param dimension - * @param dimString - * @param parent - */ - private static void addAllComplexTypeChildren(CarbonDimension dimension, StringBuilder dimString, - String parent) { - dimString.append( - dimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getDataType() - + CarbonCommonConstants.COLON_SPC_CHARACTER + parent - + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getColumnId() - + CarbonCommonConstants.HASH_SPC_CHARACTER); - for (int i = 0; i < dimension.getNumberOfChild(); i++) { - CarbonDimension childDim = dimension.getListOfChildDimensions().get(i); - if (childDim.getNumberOfChild() > 0) { - addAllComplexTypeChildren(childDim, dimString, dimension.getColName()); - } else { - dimString.append( - childDim.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim - .getDataType() + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getColName() - + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim.getColumnId() - + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim.getOrdinal() - + CarbonCommonConstants.HASH_SPC_CHARACTER); - } - } - } - - /** - * the method returns the ColumnSchemaDetailsWrapper - * - * @param dimensions - * @return - */ - public static ColumnSchemaDetailsWrapper getColumnSchemaDetails( - List dimensions) { - ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper = new ColumnSchemaDetailsWrapper(); - Map columnSchemaDetailsMap = - new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); - fillColumnSchemaDetailsWithComplex(dimensions, columnSchemaDetailsMap); - columnSchemaDetailsWrapper.setColumnSchemaDetailsMap(columnSchemaDetailsMap); - return columnSchemaDetailsWrapper; - } - - private static void fillColumnSchemaDetailsWithComplex( - List dimensions, - Map columnSchemaDetailsMap) { - for (CarbonDimension cDimension : dimensions) { - ColumnSchemaDetails details = - new ColumnSchemaDetails(cDimension.getColName(), cDimension.getDataType(), - CarbonUtil.hasEncoding(cDimension.getEncoder(), Encoding.DIRECT_DICTIONARY)); - columnSchemaDetailsMap.put(cDimension.getColumnSchema().getColumnUniqueId(), details); - if (cDimension.isComplex()) { - fillColumnSchemaDetailsWithComplex(cDimension.getListOfChildDimensions(), - columnSchemaDetailsMap); - } - } - } - - /** - * Get dimension string from a array of TableDimension,which can be shared - * TableDimension within schema or in a table. - * - * @param table - * @param dimensions - * @return - */ - public static int getNoDictionaryDimensionString(List dimensions, - StringBuilder dimString, int counter, CarbonDataLoadSchema carbonDataLoadSchema) { - for (CarbonDimension cDimension : dimensions) { - if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) { - continue; - } - - String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema); - dimString.append( - tableName + '_' + cDimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER - + counter + CarbonCommonConstants.COLON_SPC_CHARACTER + -1 - + CarbonCommonConstants.COLON_SPC_CHARACTER + 'Y' - + CarbonCommonConstants.COMA_SPC_CHARACTER); - counter++; - } - return counter; - } - - public static String getColumnPropertiesString(List dimensions) { - StringBuilder colPropertiesString = new StringBuilder(); - for (int dim = 0; dim < dimensions.size(); dim++) { - CarbonDimension dimension = dimensions.get(dim); - if (dimension.isComplex()) { - List childs = dimension.getListOfChildDimensions(); - for (CarbonDimension child : childs) { - buildDimensionColumnPropertyString(child, colPropertiesString, dim); - } - } else { - buildDimensionColumnPropertyString(dimension, colPropertiesString, dim); - } - - } - - return colPropertiesString.toString(); - } - - protected static void buildDimensionColumnPropertyString(CarbonDimension dimension, - StringBuilder colPropertiesString, int dim) { - Map columnProperties = dimension.getColumnProperties(); - if (null != columnProperties && columnProperties.size() > 0) { - if (colPropertiesString.length() > 0) { - colPropertiesString.append(CarbonCommonConstants.HASH_SPC_CHARACTER); - } - colPropertiesString.append(dimension.getColName()) - .append(CarbonCommonConstants.COLON_SPC_CHARACTER); - int size = columnProperties.entrySet().size(); - int index = 0; - Iterator> itr = columnProperties.entrySet().iterator(); - while (itr.hasNext()) { - Entry entry = itr.next(); - colPropertiesString.append(entry.getKey()) - .append(CarbonCommonConstants.HYPHEN_SPC_CHARACTER); - colPropertiesString.append(entry.getValue()); - index++; - if (index < size) { - colPropertiesString.append(CarbonCommonConstants.COMA_SPC_CHARACTER); - } - } - } - } -} diff --git a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java b/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java deleted file mode 100644 index 15b0ac8e9a5..00000000000 --- a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.pentaho.di.trans.steps.hadoopfileinput; - -import org.pentaho.di.core.annotations.Step; -import org.pentaho.di.trans.steps.textfileinput.TextFileInputMeta; - -@Step(id = "HadoopFileInputPlugin", image = "HDI.png", name = "HadoopFileInputPlugin", - description = "Process files from an HDFS location", categoryDescription = "Hadoop") -public class HadoopFileInputMeta extends TextFileInputMeta { - -} diff --git a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties b/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties deleted file mode 100644 index 4ed60dd4a5a..00000000000 --- a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties +++ /dev/null @@ -1,18 +0,0 @@ -# -# 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. -# -HadoopFileInputDialog.DialogTitle=Hadoop File Input \ No newline at end of file diff --git a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java index 714a32fd44c..156c57eadb4 100644 --- a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java +++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java @@ -40,7 +40,7 @@ import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.test.util.StoreCreator; +import org.apache.carbondata.processing.StoreCreator; import junit.framework.TestCase; import org.junit.AfterClass; diff --git a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java index 4bb7d165b16..68d822b0b33 100644 --- a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java +++ b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java @@ -20,9 +20,9 @@ import org.apache.carbondata.core.locks.LocalFileLock; import org.apache.carbondata.core.locks.LockUsage; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.pentaho.di.core.util.Assert; /** * Test class to test the functionality of the local file locking. diff --git a/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java similarity index 79% rename from processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java rename to processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java index 78e7e8352bb..85f8470aa70 100644 --- a/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java +++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java @@ -14,19 +14,39 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.carbondata.test.util; - -import com.google.gson.Gson; +package org.apache.carbondata.processing; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.Charset; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.UUID; + +import org.apache.carbondata.common.CarbonIterator; 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.CarbonCommonConstants; +import org.apache.carbondata.core.datastore.impl.FileFactory; +import org.apache.carbondata.core.fileoperations.AtomicFileOperations; +import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl; +import org.apache.carbondata.core.fileoperations.FileWriteOperation; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.processing.model.CarbonDataLoadSchema; +import org.apache.carbondata.core.metadata.CarbonMetadata; import org.apache.carbondata.core.metadata.CarbonTableIdentifier; import org.apache.carbondata.core.metadata.ColumnIdentifier; -import org.apache.carbondata.core.metadata.CarbonMetadata; import org.apache.carbondata.core.metadata.converter.SchemaConverter; import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl; import org.apache.carbondata.core.metadata.datatype.DataType; @@ -40,14 +60,11 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.util.path.CarbonStorePath; -import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.processing.csvreaderstep.BlockDetails; import org.apache.carbondata.core.statusmanager.LoadMetadataDetails; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.path.CarbonStorePath; +import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.core.writer.CarbonDictionaryWriter; import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl; import org.apache.carbondata.core.writer.ThriftWriter; @@ -55,22 +72,25 @@ import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl; import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo; import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator; -import org.apache.carbondata.core.fileoperations.AtomicFileOperations; -import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl; -import org.apache.carbondata.core.fileoperations.FileWriteOperation; -import org.apache.carbondata.processing.api.dataloader.DataLoadModel; import org.apache.carbondata.processing.api.dataloader.SchemaInfo; -import org.apache.carbondata.processing.csvload.DataGraphExecuter; -import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus; -import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus; -import org.apache.carbondata.processing.graphgenerator.GraphGenerator; -import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException; -import org.apache.hadoop.fs.Path; +import org.apache.carbondata.processing.constants.TableOptionConstant; +import org.apache.carbondata.processing.csvload.BlockDetails; +import org.apache.carbondata.processing.csvload.CSVInputFormat; +import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator; +import org.apache.carbondata.processing.csvload.StringArrayWritable; +import org.apache.carbondata.processing.model.CarbonDataLoadSchema; +import org.apache.carbondata.processing.model.CarbonLoadModel; +import org.apache.carbondata.processing.newflow.DataLoadExecutor; +import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants; -import java.io.*; -import java.nio.charset.Charset; -import java.text.SimpleDateFormat; -import java.util.*; +import com.google.gson.Gson; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.mapred.TaskAttemptID; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; /** * This class will create store file based on provided schema @@ -109,20 +129,46 @@ public static void createCarbonStore() { CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, absoluteTableIdentifier.getStorePath()); - String kettleHomePath = "../processing/carbonplugins"; CarbonTable table = createTable(); writeDictionary(factFilePath, table); CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table); - LoadModel loadModel = new LoadModel(); + CarbonLoadModel loadModel = new CarbonLoadModel(); String partitionId = "0"; - loadModel.setSchema(schema); + loadModel.setCarbonDataLoadSchema(schema); loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName()); loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName()); loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName()); loadModel.setFactFilePath(factFilePath); loadModel.setLoadMetadataDetails(new ArrayList()); + loadModel.setStorePath(absoluteTableIdentifier.getStorePath()); + loadModel.setDateFormat(null); + loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty( + CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, + CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)); + loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty( + CarbonCommonConstants.CARBON_DATE_FORMAT, + CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)); + loadModel + .setSerializationNullFormat( + TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N"); + loadModel + .setBadRecordsLoggerEnable( + TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false"); + loadModel + .setBadRecordsAction( + TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE"); + loadModel + .setIsEmptyDataBadRecord( + DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false"); + loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary"); + loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(",")); + loadModel.setTaskNo("0"); + loadModel.setSegmentId("0"); + loadModel.setPartitionId("0"); + loadModel.setFactTimeStamp(System.currentTimeMillis()); + + executeGraph(loadModel, absoluteTableIdentifier.getStorePath()); - executeGraph(loadModel, absoluteTableIdentifier.getStorePath(), kettleHomePath); } catch (Exception e) { e.printStackTrace(); } @@ -310,12 +356,10 @@ private static void writeDictionary(String factFilePath, CarbonTable table) thro * * @param loadModel * @param storeLocation - * @param kettleHomePath * @throws Exception */ - public static void executeGraph(LoadModel loadModel, String storeLocation, String kettleHomePath) + public static void executeGraph(CarbonLoadModel loadModel, String storeLocation) throws Exception { - System.setProperty("KETTLE_HOME", kettleHomePath); new File(storeLocation).mkdirs(); String outPutLoc = storeLocation + "/etl"; String databaseName = loadModel.getDatabaseName(); @@ -343,29 +387,39 @@ public static void executeGraph(LoadModel loadModel, String storeLocation, Strin path.delete(); } - DataProcessTaskStatus dataProcessTaskStatus = new DataProcessTaskStatus(databaseName, tableName); - dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath()); SchemaInfo info = new SchemaInfo(); BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()), 0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"}); - GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails }); - dataProcessTaskStatus.setBlocksID("qwqwq"); - dataProcessTaskStatus.setEscapeCharacter("\\"); - dataProcessTaskStatus.setQuoteCharacter("\""); - dataProcessTaskStatus.setCommentCharacter("#"); - dataProcessTaskStatus.setDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT); + Configuration configuration = new Configuration(); + CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar()); + CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter()); + CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar()); + CSVInputFormat.setHeaderExtractionEnabled(configuration, true); + CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar()); + CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance() + .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE, + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT)); + + TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0)); + CSVInputFormat format = new CSVInputFormat(); + + RecordReader recordReader = + format.createRecordReader(blockDetails, hadoopAttemptContext); + + CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext); + new DataLoadExecutor().execute(loadModel, + storeLocation, + new CarbonIterator[]{readerIterator}); + info.setDatabaseName(databaseName); info.setTableName(tableName); - generateGraph(dataProcessTaskStatus, info, loadModel.getTableName(), "0", loadModel.getSchema(), null, - loadModel.getLoadMetadataDetails()); - - DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus); - graphExecuter - .executeGraph(graphPath, info, loadModel.getSchema()); +// DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus); +// graphExecuter +// .executeGraph(graphPath, info, loadModel.getSchema()); // LoadMetadataDetails[] loadDetails = // CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath()); - writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(), + writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(), new ArrayList()); String segLocation = @@ -436,40 +490,6 @@ public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databas } - /** - * generate graph - * - * @param dataProcessTaskStatus - * @param info - * @param tableName - * @param partitionID - * @param schema - * @param factStoreLocation - * @param loadMetadataDetails - * @throws GraphGeneratorException - */ - private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info, - String tableName, String partitionID, CarbonDataLoadSchema schema, String factStoreLocation, - List loadMetadataDetails) - throws GraphGeneratorException { - DataLoadModel model = new DataLoadModel(); - model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath() || null != dataProcessTaskStatus.getFilesToProcess()); - model.setSchemaInfo(info); - model.setTableName(dataProcessTaskStatus.getTableName()); - model.setTaskNo("1"); - model.setBlocksID(dataProcessTaskStatus.getBlocksID()); - model.setFactTimeStamp(System.currentTimeMillis() + ""); - model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter()); - model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter()); - model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter()); - model.setDateFormat(dataProcessTaskStatus.getDateFormat()); - String outputLocation = CarbonProperties.getInstance() - .getProperty("store_output_location", "../carbon-store/system/carbon/etl"); - GraphGenerator generator = - new GraphGenerator(model, partitionID, factStoreLocation, schema, "0", outputLocation); - generator.generateGraph(); - } - public static String readCurrentTime() { SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP); String date = null; @@ -479,59 +499,6 @@ public static String readCurrentTime() { return date; } - /** - * This is local model object used inside this class to store information related to data loading - */ - private static class LoadModel { - - private CarbonDataLoadSchema schema; - private String tableName; - private String databaseName; - private List loadMetaDetail; - private String factFilePath; - - public void setSchema(CarbonDataLoadSchema schema) { - this.schema = schema; - } - - public List getLoadMetadataDetails() { - return loadMetaDetail; - } - - public CarbonDataLoadSchema getSchema() { - return schema; - } - - public String getFactFilePath() { - return factFilePath; - } - - public String getTableName() { - return tableName; - } - - public String getDatabaseName() { - return databaseName; - } - - public void setLoadMetadataDetails(List loadMetaDetail) { - this.loadMetaDetail = loadMetaDetail; - } - - public void setFactFilePath(String factFilePath) { - this.factFilePath = factFilePath; - } - - public void setTableName(String tableName) { - this.tableName = tableName; - } - - public void setDatabaseName(String databaseName) { - this.databaseName = databaseName; - } - - } - public static void main(String[] args) { StoreCreator.createCarbonStore(); }