Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 06276200C55 for ; Thu, 30 Mar 2017 07:12:34 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 048A6160B9D; Thu, 30 Mar 2017 05:12:34 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 9D1B3160B8A for ; Thu, 30 Mar 2017 07:12:31 +0200 (CEST) Received: (qmail 78201 invoked by uid 500); 30 Mar 2017 05:12:30 -0000 Mailing-List: contact commits-help@carbondata.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@carbondata.incubator.apache.org Delivered-To: mailing list commits@carbondata.incubator.apache.org Received: (qmail 78192 invoked by uid 99); 30 Mar 2017 05:12:30 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 Mar 2017 05:12:30 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id 2B845C0145 for ; Thu, 30 Mar 2017 05:12:30 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.222 X-Spam-Level: X-Spam-Status: No, score=-4.222 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id 5VLDnZjslK37 for ; Thu, 30 Mar 2017 05:12:17 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 73B6C60CF7 for ; Thu, 30 Mar 2017 05:12:10 +0000 (UTC) Received: (qmail 77747 invoked by uid 99); 30 Mar 2017 05:12:09 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 Mar 2017 05:12:09 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 53BC2E9652; Thu, 30 Mar 2017 05:12:09 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jackylk@apache.org To: commits@carbondata.incubator.apache.org Date: Thu, 30 Mar 2017 05:12:16 -0000 Message-Id: <5d1bf355ac2f4c93b1605c5194506aaa@git.apache.org> In-Reply-To: <012add92fc2a4ddb8f1e0e99b67ca614@git.apache.org> References: <012add92fc2a4ddb8f1e0e99b67ca614@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [08/13] incubator-carbondata git commit: Removed kettle related code and refactored archived-at: Thu, 30 Mar 2017 05:12:34 -0000 http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java ---------------------------------------------------------------------- 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 cb6c432..0000000 --- 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; - } -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml ---------------------------------------------------------------------- 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 32b1452..0000000 --- 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 http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java ---------------------------------------------------------------------- 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 eba739a..0000000 --- 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; } -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java ---------------------------------------------------------------------- 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 412c1ff..0000000 --- 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(); -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java ---------------------------------------------------------------------- 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 ea7945f..0000000 --- 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; - } - -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java ---------------------------------------------------------------------- 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 68ca84c..0000000 --- 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; - } - -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java ---------------------------------------------------------------------- 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 442c0f0..02ceb06 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.nio.ByteBuffer; 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 class ArrayDataType implements GenericDataType { } - /* - * 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 { http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java ---------------------------------------------------------------------- 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 91caf7a..6b54d2d 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 java.util.List; 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 & @@ -57,21 +54,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 * @throws IOException http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java ---------------------------------------------------------------------- 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 105f5f4..e7e48e9 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.core.util.DataTypeUtil; 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 class PrimitiveDataType implements GenericDataType { 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 = http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java ---------------------------------------------------------------------- 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 a131004..a61144e 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.nio.ByteBuffer; 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 class StructDataType implements GenericDataType { } - /* - * 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()); http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java ---------------------------------------------------------------------- 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 7552e6a..0000000 --- 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; - } -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java ---------------------------------------------------------------------- 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 2876eb8..0000000 --- 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()])); - } -} http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java ---------------------------------------------------------------------- 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 95a7823..0000000 --- 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() { - - } - -}