From commits-return-9584-archive-asf-public=cust-asf.ponee.io@carbondata.apache.org Sun Mar 4 13:24:27 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 8F023180778 for ; Sun, 4 Mar 2018 13:24:24 +0100 (CET) Received: (qmail 13916 invoked by uid 500); 4 Mar 2018 12:24:23 -0000 Mailing-List: contact commits-help@carbondata.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@carbondata.apache.org Delivered-To: mailing list commits@carbondata.apache.org Received: (qmail 13845 invoked by uid 99); 4 Mar 2018 12:24:23 -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; Sun, 04 Mar 2018 12:24:23 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8ACEBF4EB5; Sun, 4 Mar 2018 12:24:22 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jackylk@apache.org To: commits@carbondata.apache.org Date: Sun, 04 Mar 2018 12:24:43 -0000 Message-Id: <2ec4c52559bd4f2d95a52d0287fda421@git.apache.org> In-Reply-To: <91fce89f58fa4e968657a8eadba45632@git.apache.org> References: <91fce89f58fa4e968657a8eadba45632@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [23/50] [abbrv] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method Refactory CarbonTablePath: 1.Remove CarbonStorePath and use CarbonTablePath only. 2.Make CarbonTablePath an utility without object creation, it can avoid creating object before using it, thus code is cleaner and GC is less. This closes #1768 Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/9b9125b6 Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/9b9125b6 Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/9b9125b6 Branch: refs/heads/carbonstore-rebase5 Commit: 9b9125b6dfa4373364faca9b605569fb2d775c17 Parents: 55c4e43 Author: Jacky Li Authored: Wed Jan 31 16:14:27 2018 +0800 Committer: Jacky Li Committed: Sun Mar 4 19:31:36 2018 +0800 ---------------------------------------------------------------------- .../DictionaryColumnUniqueIdentifier.java | 29 +- .../dictionary/ManageDictionaryAndBTree.java | 13 +- .../core/metadata/AbsoluteTableIdentifier.java | 4 +- .../core/metadata/SegmentFileStore.java | 3 +- .../core/metadata/schema/table/CarbonTable.java | 11 +- .../core/mutate/CarbonUpdateUtil.java | 58 ++-- .../core/scan/executor/util/QueryUtil.java | 7 +- .../scan/executor/util/RestructureUtil.java | 6 +- .../scan/filter/FilterExpressionProcessor.java | 2 +- .../filter/executer/FalseFilterExecutor.java | 8 +- .../RowLevelRangeGrtThanFiterExecuterImpl.java | 1 + ...elRangeGrtrThanEquaToFilterExecuterImpl.java | 1 + ...velRangeLessThanEqualFilterExecuterImpl.java | 1 + .../RowLevelRangeLessThanFiterExecuterImpl.java | 1 + .../FalseConditionalResolverImpl.java | 4 +- .../core/service/CarbonCommonFactory.java | 16 - .../carbondata/core/service/PathService.java | 35 --- .../core/service/impl/PathFactory.java | 50 ---- .../statusmanager/SegmentStatusManager.java | 31 +- .../SegmentUpdateStatusManager.java | 70 ++--- .../apache/carbondata/core/util/CarbonUtil.java | 98 ++---- .../util/path/CarbonSharedDictionaryPath.java | 71 ----- .../core/util/path/CarbonStorePath.java | 71 ----- .../core/util/path/CarbonTablePath.java | 298 ++++++------------- .../dictionary/AbstractDictionaryCacheTest.java | 11 +- .../dictionary/ForwardDictionaryCacheTest.java | 6 +- .../dictionary/ReverseDictionaryCacheTest.java | 6 +- .../reader/CarbonDictionaryReaderImplTest.java | 8 - .../CarbonFormatDirectoryStructureTest.java | 18 +- .../path/CarbonFormatSharedDictionaryTest.java | 44 --- .../writer/CarbonDictionaryWriterImplTest.java | 19 +- .../CarbonBatchSparkStreamingExample.scala | 9 +- .../CarbonStreamSparkStreamingExample.scala | 10 +- .../CarbonStructuredStreamingExample.scala | 11 +- ...CarbonStructuredStreamingWithRowParser.scala | 9 +- .../hadoop/api/CarbonTableInputFormat.java | 8 +- .../streaming/CarbonStreamRecordWriter.java | 6 +- .../carbondata/hadoop/util/SchemaReader.java | 18 +- .../hadoop/test/util/StoreCreator.java | 16 +- .../presto/CarbondataRecordSetProvider.java | 7 +- .../presto/impl/CarbonTableCacheModel.java | 13 +- .../presto/impl/CarbonTableReader.java | 49 ++- .../presto/util/CarbonDataStoreCreator.scala | 10 +- .../sdv/generated/MergeIndexTestCase.scala | 19 +- .../dataload/TestLoadDataGeneral.scala | 11 +- .../InsertIntoCarbonTableTestCase.scala | 8 +- .../createTable/TestCreateTableAsSelect.scala | 2 +- .../datacompaction/DataCompactionLockTest.scala | 6 +- .../MajorCompactionIgnoreInMinorTest.scala | 12 +- .../dataload/TestBatchSortDataLoad.scala | 5 +- .../dataload/TestDataLoadWithFileName.scala | 5 +- .../dataload/TestGlobalSortDataLoad.scala | 8 +- .../dataretention/DataRetentionTestCase.scala | 6 +- .../TestDataLoadingForPartitionTable.scala | 4 +- .../StandardPartitionTableLoadingTestCase.scala | 6 +- .../org/apache/carbondata/api/CarbonStore.scala | 4 +- .../carbondata/spark/CarbonSparkFactory.scala | 2 +- .../spark/DictionaryDetailHelper.scala | 9 +- .../spark/rdd/AlterTableAddColumnRDD.scala | 17 +- .../spark/rdd/CarbonGlobalDictionaryRDD.scala | 6 - .../carbondata/spark/util/CommonUtil.scala | 30 +- .../carbondata/spark/util/DataLoadingUtil.scala | 10 +- .../spark/util/GlobalDictionaryUtil.scala | 16 +- .../command/carbonTableSchemaCommon.scala | 9 +- .../org/apache/spark/util/PartitionUtils.scala | 6 +- .../spark/rdd/AggregateDataMapCompactor.scala | 19 +- .../spark/rdd/CarbonDataRDDFactory.scala | 14 +- .../spark/rdd/CarbonTableCompactor.scala | 8 +- .../CarbonAlterTableCompactionCommand.scala | 13 +- .../management/CarbonLoadDataCommand.scala | 12 +- .../management/CarbonShowLoadsCommand.scala | 2 +- .../management/RefreshCarbonTableCommand.scala | 17 +- .../command/mutation/DeleteExecution.scala | 7 +- .../CarbonAlterTableDropPartitionCommand.scala | 16 +- .../CarbonAlterTableSplitPartitionCommand.scala | 20 +- .../CreatePreAggregateTableCommand.scala | 2 +- .../preaaggregate/PreAggregateListeners.scala | 21 +- .../preaaggregate/PreAggregateUtil.scala | 35 ++- .../CarbonAlterTableAddColumnCommand.scala | 8 +- .../CarbonAlterTableDataTypeChangeCommand.scala | 14 +- .../CarbonAlterTableDropColumnCommand.scala | 6 +- .../schema/CarbonAlterTableRenameCommand.scala | 30 +- .../spark/sql/hive/CarbonFileMetastore.scala | 27 +- .../spark/sql/hive/CarbonHiveMetaStore.scala | 13 +- .../apache/spark/sql/hive/CarbonMetaStore.scala | 4 +- .../apache/spark/sql/hive/CarbonRelation.scala | 11 +- .../org/apache/spark/util/AlterTableUtil.scala | 54 ++-- .../partition/TestAlterPartitionTable.scala | 4 +- .../spark/util/AllDictionaryTestCase.scala | 6 +- .../spark/util/DictionaryTestCaseUtil.scala | 2 +- .../util/ExternalColumnDictionaryTestCase.scala | 6 +- .../TestStreamingTableOperation.scala | 26 +- .../TestStreamingTableWithRowParser.scala | 9 +- .../restructure/AlterTableRevertTestCase.scala | 2 +- .../loading/TableProcessingOperations.java | 59 ++-- .../converter/impl/FieldEncoderFactory.java | 9 +- .../merger/AbstractResultProcessor.java | 8 +- .../processing/merger/CarbonCompactionUtil.java | 2 +- .../processing/merger/CarbonDataMergerUtil.java | 87 ++---- .../merger/CompactionResultSortProcessor.java | 3 +- .../merger/RowResultMergerProcessor.java | 3 +- .../store/CarbonFactDataHandlerModel.java | 6 +- .../util/CarbonDataProcessorUtil.java | 13 +- .../processing/util/CarbonLoaderUtil.java | 95 +++--- .../processing/util/DeleteLoadFolders.java | 8 +- .../carbon/datastore/BlockIndexStoreTest.java | 158 +--------- .../carbondata/processing/StoreCreator.java | 34 +-- .../streaming/segment/StreamSegment.java | 24 +- .../carbondata/streaming/StreamHandoffRDD.scala | 22 +- .../streaming/StreamSinkFactory.scala | 14 +- .../streaming/CarbonAppendableStreamSink.scala | 18 +- 111 files changed, 725 insertions(+), 1614 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java index f5652ac..a7b47c9 100644 --- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java +++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java @@ -22,7 +22,6 @@ import java.io.Serializable; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.ColumnIdentifier; import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; /** @@ -42,8 +41,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable { */ private ColumnIdentifier columnIdentifier; - private transient CarbonTablePath carbonTablePath; - private DataType dataType; private String dictionaryLocation; @@ -68,8 +65,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable { this.columnIdentifier = columnIdentifier; this.dataType = columnIdentifier.getDataType(); this.dictionaryLocation = - CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier) - .getMetadataDirectoryPath(); + CarbonTablePath.getMetadataPath(dictionarySourceAbsoluteTableIdentifier.getTablePath()); } /** @@ -100,13 +96,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable { return dataType; } - public CarbonTablePath getCarbonTablePath() { - if (null == carbonTablePath) { - carbonTablePath = CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier); - } - return carbonTablePath; - } - /** * @return columnIdentifier */ @@ -118,24 +107,24 @@ public class DictionaryColumnUniqueIdentifier implements Serializable { * @return dictionary file path */ public String getDictionaryFilePath() { - return getCarbonTablePath() - .getDictionaryFilePath(dictionaryLocation, columnIdentifier.getColumnId()); + return CarbonTablePath.getExternalDictionaryFilePath( + dictionaryLocation, columnIdentifier.getColumnId()); } /** * @return dictionary metadata file path */ public String getDictionaryMetaFilePath() { - return getCarbonTablePath() - .getDictionaryMetaFilePath(dictionaryLocation, columnIdentifier.getColumnId()); + return CarbonTablePath.getExternalDictionaryMetaFilePath( + dictionaryLocation, columnIdentifier.getColumnId()); } /** * @return sort index file path */ public String getSortIndexFilePath() { - return getCarbonTablePath() - .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId()); + return CarbonTablePath.getExternalSortIndexFilePath( + dictionaryLocation, columnIdentifier.getColumnId()); } /** @@ -143,8 +132,8 @@ public class DictionaryColumnUniqueIdentifier implements Serializable { * @return sort index file path with given offset */ public String getSortIndexFilePath(long offset) { - return getCarbonTablePath() - .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId(), offset); + return CarbonTablePath.getExternalSortIndexFilePath( + dictionaryLocation, columnIdentifier.getColumnId(), offset); } /** http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java index 4f8f724..a7d6027 100644 --- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java +++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java @@ -36,7 +36,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; import org.apache.carbondata.core.statusmanager.LoadMetadataDetails; import org.apache.carbondata.core.statusmanager.SegmentStatusManager; -import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; /** @@ -56,13 +55,11 @@ public class ManageDictionaryAndBTree { * clear the dictionary cache * * @param columnSchema - * @param carbonTableIdentifier + * @param identifier */ public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema, - AbsoluteTableIdentifier carbonTableIdentifier) { - CarbonTablePath carbonTablePath = - CarbonStorePath.getCarbonTablePath(carbonTableIdentifier); - String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath(); + AbsoluteTableIdentifier identifier) { + String metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath()); CarbonFile metadataDir = FileFactory .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath)); if (metadataDir.exists()) { @@ -90,7 +87,7 @@ public class ManageDictionaryAndBTree { } } // remove dictionary cache - removeDictionaryColumnFromCache(carbonTableIdentifier, columnSchema.getColumnUniqueId()); + removeDictionaryColumnFromCache(identifier, columnSchema.getColumnUniqueId()); } /** @@ -101,7 +98,7 @@ public class ManageDictionaryAndBTree { public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) { // clear Btree cache from LRU cache LoadMetadataDetails[] loadMetadataDetails = - SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath()); + SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath()); String[] segments = new String[loadMetadataDetails.length]; int i = 0; for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) { http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java index d3250aa..3ea1f60 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java @@ -36,7 +36,6 @@ public class AbsoluteTableIdentifier implements Serializable { */ private String tablePath; - private boolean isLocalPath; /** * carbon table identifier which will have table name and table database @@ -47,7 +46,6 @@ public class AbsoluteTableIdentifier implements Serializable { private AbsoluteTableIdentifier(String tablePath, CarbonTableIdentifier carbonTableIdentifier) { //TODO this should be moved to common place where path handling will be handled this.tablePath = FileFactory.getUpdatedFilePath(tablePath); - isLocalPath = tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX); this.carbonTableIdentifier = carbonTableIdentifier; } @@ -79,7 +77,7 @@ public class AbsoluteTableIdentifier implements Serializable { } public String appendWithLocalPrefix(String path) { - if (isLocalPath) { + if (tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX)) { return CarbonCommonConstants.LOCAL_FILE_PREFIX + path; } else { return path; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java index 2d31b4e..3fc8ad6 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java @@ -491,8 +491,7 @@ public class SegmentFileStore { public static void cleanSegments(CarbonTable table, List partitionSpecs, boolean forceDelete) throws IOException { - LoadMetadataDetails[] details = - SegmentStatusManager.readLoadMetadata(table.getMetaDataFilepath()); + LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(table.getMetadataPath()); // scan through each segment. for (LoadMetadataDetails segment : details) { // if this segment is valid then only we will go for deletion of related http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java index d17d865..6ed1197 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java @@ -424,13 +424,20 @@ public class CarbonTable implements Serializable { } /** - * @return the metaDataFilepath + * Return the metadata path of the table */ - public String getMetaDataFilepath() { + public String getMetadataPath() { return CarbonTablePath.getMetadataPath(getTablePath()); } /** + * Return the segment path of the specified segmentId + */ + public String getSemgentPath(String segmentId) { + return CarbonTablePath.getSegmentPath(getTablePath(), segmentId); + } + + /** * @return storepath */ public String getTablePath() { http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java index 7289c98..4ff19cb 100644 --- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java @@ -43,10 +43,8 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager; import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager; 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; - /** * This class contains all update utility methods */ @@ -202,18 +200,10 @@ public class CarbonUpdateUtil { List segmentsToBeDeleted, List segmentFilesTobeUpdated) { boolean status = false; - - String metaDataFilepath = table.getMetaDataFilepath(); - - AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier(); - - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - - String tableStatusPath = carbonTablePath.getTableStatusFilePath(); - - SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier); + String metaDataFilepath = table.getMetadataPath(); + AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier(); + String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()); + SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier); ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock(); boolean lockStatus = false; @@ -225,7 +215,7 @@ public class CarbonUpdateUtil { + " for table status updation"); LoadMetadataDetails[] listOfLoadFolderDetailsArray = - segmentStatusManager.readLoadMetadata(metaDataFilepath); + SegmentStatusManager.readLoadMetadata(metaDataFilepath); for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) { @@ -312,12 +302,8 @@ public class CarbonUpdateUtil { */ public static void cleanStaleDeltaFiles(CarbonTable table, final String timeStamp) { - AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier(); - - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - String partitionDir = carbonTablePath.getPartitionDir(); + AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier(); + String partitionDir = CarbonTablePath.getPartitionDir(identifier.getTablePath()); CarbonFile file = FileFactory.getCarbonFile(partitionDir, FileFactory.getFileType(partitionDir)); if (!file.exists()) { @@ -387,8 +373,18 @@ public class CarbonUpdateUtil { .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN)); } - public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) { - String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId); + /** + * returns segment id from segment name + * + * @param segmentName + * @return + */ + public static String getSegmentId(String segmentName) { + return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1]; + } + + public static long getLatestTaskIdForSegment(String segmentId, String tablePath) { + String segmentDirPath = CarbonTablePath.getSegmentPath(tablePath, segmentId); // scan all the carbondata files and get the latest task ID. CarbonFile segment = @@ -427,11 +423,8 @@ public class CarbonUpdateUtil { SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier()); - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(), - table.getAbsoluteTableIdentifier().getCarbonTableIdentifier()); - - LoadMetadataDetails[] details = ssm.readLoadMetadata(table.getMetaDataFilepath()); + LoadMetadataDetails[] details = + SegmentStatusManager.readLoadMetadata(table.getMetadataPath()); String validUpdateStatusFile = ""; @@ -453,7 +446,8 @@ public class CarbonUpdateUtil { || segment.getSegmentStatus() == SegmentStatus.LOAD_PARTIAL_SUCCESS) { // take the list of files from this segment. - String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segment.getLoadName()); + String segmentPath = CarbonTablePath.getSegmentPath( + table.getAbsoluteTableIdentifier().getTablePath(), segment.getLoadName()); CarbonFile segDir = FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath)); CarbonFile[] allSegmentFiles = segDir.listFiles(); @@ -560,8 +554,10 @@ public class CarbonUpdateUtil { final String updateStatusTimestamp = validUpdateStatusFile .substring(validUpdateStatusFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1); - CarbonFile metaFolder = FileFactory.getCarbonFile(carbonTablePath.getMetadataDirectoryPath(), - FileFactory.getFileType(carbonTablePath.getMetadataDirectoryPath())); + String tablePath = table.getAbsoluteTableIdentifier().getTablePath(); + CarbonFile metaFolder = FileFactory.getCarbonFile( + CarbonTablePath.getMetadataPath(tablePath), + FileFactory.getFileType(CarbonTablePath.getMetadataPath(tablePath))); CarbonFile[] invalidUpdateStatusFiles = metaFolder.listFiles(new CarbonFileFilter() { @Override public boolean accept(CarbonFile file) { http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java ---------------------------------------------------------------------- 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 00cdfe7..1765efa 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 @@ -63,7 +63,6 @@ import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnRes import org.apache.carbondata.core.scan.model.ProjectionDimension; import org.apache.carbondata.core.scan.model.ProjectionMeasure; 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.commons.lang3.ArrayUtils; @@ -409,7 +408,7 @@ public class QueryUtil { } public static AbsoluteTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension, - AbsoluteTableIdentifier absoluteTableIdentifier) { + AbsoluteTableIdentifier identifier) { RelationIdentifier relation = carbonDimension.getColumnSchema() .getParentColumnTableRelations() .get(0) @@ -417,8 +416,8 @@ public class QueryUtil { String parentTableName = relation.getTableName(); String parentDatabaseName = relation.getDatabaseName(); String parentTableId = relation.getTableId(); - CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier); - String newTablePath = CarbonUtil.getNewTablePath(carbonTablePath, parentTableName); + String newTablePath = + CarbonTablePath.getNewTablePath(identifier.getTablePath(), parentTableName); return AbsoluteTableIdentifier.from(newTablePath, parentDatabaseName, parentTableName, parentTableId); } http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java index 2712cbc..e67d822 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java @@ -80,7 +80,7 @@ public class RestructureUtil { if (queryDimension.getDimension().hasEncoding(Encoding.IMPLICIT)) { presentDimension.add(queryDimension); isDimensionExists[dimIndex] = true; - dimensionInfo.dataType[queryDimension.getQueryOrder()] = + dimensionInfo.dataType[queryDimension.getOrdinal()] = queryDimension.getDimension().getDataType(); } else { for (CarbonDimension tableDimension : tableBlockDimensions) { @@ -95,7 +95,7 @@ public class RestructureUtil { currentBlockDimension.setOrdinal(queryDimension.getOrdinal()); presentDimension.add(currentBlockDimension); isDimensionExists[dimIndex] = true; - dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] = + dimensionInfo.dataType[currentBlockDimension.getOrdinal()] = currentBlockDimension.getDimension().getDataType(); break; } @@ -113,7 +113,7 @@ public class RestructureUtil { currentBlockDimension.setOrdinal(queryDimension.getOrdinal()); presentDimension.add(currentBlockDimension); isDimensionExists[dimIndex] = true; - dimensionInfo.dataType[currentBlockDimension.getQueryOrder()] = + dimensionInfo.dataType[currentBlockDimension.getOrdinal()] = currentBlockDimension.getDimension().getDataType(); break; } http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java index b882b51..26b202f 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java @@ -398,7 +398,7 @@ public class FilterExpressionProcessor implements FilterProcessor { ConditionalExpression condExpression = null; switch (filterExpressionType) { case FALSE: - return new FalseConditionalResolverImpl(expression, false, false, tableIdentifier); + return new FalseConditionalResolverImpl(expression, false, false); case TRUE: return new TrueConditionalResolverImpl(expression, false, false); case EQUALS: http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java index 2d2a15c..1c918bc 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java @@ -21,7 +21,7 @@ import java.util.BitSet; import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.intf.RowIntf; -import org.apache.carbondata.core.scan.processor.BlocksChunkHolder; +import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks; import org.apache.carbondata.core.util.BitSetGroup; /** @@ -33,9 +33,9 @@ import org.apache.carbondata.core.util.BitSetGroup; public class FalseFilterExecutor implements FilterExecuter { @Override - public BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeline) + public BitSetGroup applyFilter(RawBlockletColumnChunks rawChunks, boolean useBitsetPipeline) throws FilterUnsupportedException, IOException { - int numberOfPages = blocksChunkHolder.getDataBlock().numberOfPages(); + int numberOfPages = rawChunks.getDataBlock().numberOfPages(); BitSetGroup group = new BitSetGroup(numberOfPages); for (int i = 0; i < numberOfPages; i++) { BitSet set = new BitSet(); @@ -54,7 +54,7 @@ public class FalseFilterExecutor implements FilterExecuter { return new BitSet(); } - @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException { + @Override public void readColumnChunks(RawBlockletColumnChunks blockChunkHolder) { // Do Nothing } } http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java index 1f63a81..e35bb8a 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java @@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; 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.scan.expression.Expression; +import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java index 9140a11..d48abf9 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java @@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; 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.scan.expression.Expression; +import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java index 120671f..d89a488 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java @@ -35,6 +35,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; 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.scan.expression.Expression; +import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java index 547ecaa..a00c7db 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java @@ -35,6 +35,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding; 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.scan.expression.Expression; +import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException; import org.apache.carbondata.core.scan.filter.FilterUtil; import org.apache.carbondata.core.scan.filter.intf.RowIntf; import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java index eccda1e..63ce790 100644 --- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java +++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java @@ -28,8 +28,8 @@ public class FalseConditionalResolverImpl extends ConditionalFilterResolverImpl private static final long serialVersionUID = 4599541011924324041L; public FalseConditionalResolverImpl(Expression exp, boolean isExpressionResolve, - boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) { - super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false); + boolean isIncludeFilter) { + super(exp, isExpressionResolve, isIncludeFilter, false); } @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier, http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java index 6fd9735..3ddf00b 100644 --- a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java +++ b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java @@ -16,9 +16,7 @@ */ package org.apache.carbondata.core.service; -import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator; import org.apache.carbondata.core.service.impl.DictionaryFactory; -import org.apache.carbondata.core.service.impl.PathFactory; /** * Interface to get services @@ -32,18 +30,4 @@ public class CarbonCommonFactory { return DictionaryFactory.getInstance(); } - /** - * @return path service - */ - public static PathService getPathService() { - return PathFactory.getInstance(); - } - - /** - * @return unique id generator - */ - public static ColumnUniqueIdService getColumnUniqueIdGenerator() { - return ColumnUniqueIdGenerator.getInstance(); - } - } http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/service/PathService.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/service/PathService.java b/core/src/main/java/org/apache/carbondata/core/service/PathService.java deleted file mode 100644 index 664d045..0000000 --- a/core/src/main/java/org/apache/carbondata/core/service/PathService.java +++ /dev/null @@ -1,35 +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.service; - -import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier; -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.core.util.path.CarbonTablePath; - -/** - * Create helper to get path details - */ -public interface PathService { - - /** - * @param absoluteTableIdentifier - * @param dictionaryColumnUniqueIdentifier - * @return store path related to tables - */ - CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier absoluteTableIdentifier, - DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier); -} http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java b/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java deleted file mode 100644 index cb4ca25..0000000 --- a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java +++ /dev/null @@ -1,50 +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.service.impl; - -import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier; -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.core.service.PathService; -import org.apache.carbondata.core.util.path.CarbonStorePath; -import org.apache.carbondata.core.util.path.CarbonTablePath; - -/** - * Create helper to get path details - */ -public class PathFactory implements PathService { - - private static PathService pathService = new PathFactory(); - - /** - * @param absoluteTableIdentifier - * @param - * @return store path related to tables - */ - @Override public CarbonTablePath getCarbonTablePath( - AbsoluteTableIdentifier absoluteTableIdentifier, - DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) { - if (null != dictionaryColumnUniqueIdentifier && null != dictionaryColumnUniqueIdentifier - .getCarbonTablePath()) { - return dictionaryColumnUniqueIdentifier.getCarbonTablePath(); - } - return CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier); - } - - public static PathService getInstance() { - return pathService; - } -} http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java index 49d494a..d3e04ca 100755 --- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java +++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java @@ -47,7 +47,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.mutate.CarbonUpdateUtil; import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; import com.google.gson.Gson; @@ -60,10 +59,10 @@ public class SegmentStatusManager { private static final LogService LOG = LogServiceFactory.getLogService(SegmentStatusManager.class.getName()); - private AbsoluteTableIdentifier absoluteTableIdentifier; + private AbsoluteTableIdentifier identifier; - public SegmentStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) { - this.absoluteTableIdentifier = absoluteTableIdentifier; + public SegmentStatusManager(AbsoluteTableIdentifier identifier) { + this.identifier = identifier; } /** @@ -72,7 +71,7 @@ public class SegmentStatusManager { * @return */ public ICarbonLock getTableStatusLock() { - return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, LockUsage.TABLE_STATUS_LOCK); + return CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK); } /** @@ -80,9 +79,7 @@ public class SegmentStatusManager { */ public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier) throws IOException { - String tableStatusPath = CarbonStorePath - .getCarbonTablePath(identifier.getTablePath(), identifier.getCarbonTableIdentifier()) - .getTableStatusFilePath(); + String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()); if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) { return 0L; } else { @@ -114,13 +111,11 @@ public class SegmentStatusManager { List listOfInvalidSegments = new ArrayList<>(10); List listOfStreamSegments = new ArrayList<>(10); List listOfInProgressSegments = new ArrayList<>(10); - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); try { if (loadMetadataDetails == null) { - loadMetadataDetails = readTableStatusFile(carbonTablePath.getTableStatusFilePath()); + loadMetadataDetails = readTableStatusFile( + CarbonTablePath.getTableStatusFilePath(identifier.getTablePath())); } //just directly iterate Array for (LoadMetadataDetails segment : loadMetadataDetails) { @@ -306,9 +301,7 @@ public class SegmentStatusManager { if (carbonDeleteSegmentLock.lockWithRetries()) { LOG.info("Delete segment lock has been successfully acquired"); - CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath( - identifier.getTablePath(), identifier.getCarbonTableIdentifier()); - String dataLoadLocation = carbonTablePath.getTableStatusFilePath(); + String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()); LoadMetadataDetails[] listOfLoadFolderDetailsArray = null; if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) { // log error. @@ -388,9 +381,7 @@ public class SegmentStatusManager { if (carbonDeleteSegmentLock.lockWithRetries()) { LOG.info("Delete segment lock has been successfully acquired"); - CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath( - identifier.getTablePath(), identifier.getCarbonTableIdentifier()); - String dataLoadLocation = carbonTablePath.getTableStatusFilePath(); + String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()); LoadMetadataDetails[] listOfLoadFolderDetailsArray = null; if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) { @@ -718,7 +709,7 @@ public class SegmentStatusManager { return false; } boolean loadInProgress = false; - String metaPath = carbonTable.getMetaDataFilepath(); + String metaPath = carbonTable.getMetadataPath(); LoadMetadataDetails[] listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath); if (listOfLoadFolderDetailsArray.length != 0) { @@ -743,7 +734,7 @@ public class SegmentStatusManager { return false; } boolean loadInProgress = false; - String metaPath = carbonTable.getMetaDataFilepath(); + String metaPath = carbonTable.getMetadataPath(); LoadMetadataDetails[] listOfLoadFolderDetailsArray = SegmentStatusManager.readLoadMetadata(metaPath); if (listOfLoadFolderDetailsArray.length != 0) { http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java index 92af9a1..94a4243 100644 --- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java +++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java @@ -44,12 +44,12 @@ import org.apache.carbondata.core.locks.CarbonLockFactory; import org.apache.carbondata.core.locks.ICarbonLock; import org.apache.carbondata.core.locks.LockUsage; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.mutate.CarbonUpdateUtil; import org.apache.carbondata.core.mutate.SegmentUpdateDetails; import org.apache.carbondata.core.mutate.TupleIdEnum; import org.apache.carbondata.core.mutate.UpdateVO; import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; import com.google.gson.Gson; @@ -65,18 +65,15 @@ public class SegmentUpdateStatusManager { private static final LogService LOG = LogServiceFactory.getLogService(SegmentUpdateStatusManager.class.getName()); - private AbsoluteTableIdentifier absoluteTableIdentifier; + private AbsoluteTableIdentifier identifier; private LoadMetadataDetails[] segmentDetails; private SegmentUpdateDetails[] updateDetails; - private CarbonTablePath carbonTablePath; private Map blockAndDetailsMap; private boolean isPartitionTable; public SegmentUpdateStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] segmentDetails) { - this.absoluteTableIdentifier = absoluteTableIdentifier; - carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); + this.identifier = absoluteTableIdentifier; // current it is used only for read function scenarios, as file update always requires to work // on latest file status. this.segmentDetails = segmentDetails; @@ -88,16 +85,14 @@ public class SegmentUpdateStatusManager { } /** - * @param absoluteTableIdentifier + * @param identifier */ - public SegmentUpdateStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) { - this.absoluteTableIdentifier = absoluteTableIdentifier; - carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); + public SegmentUpdateStatusManager(AbsoluteTableIdentifier identifier) { + this.identifier = identifier; // current it is used only for read function scenarios, as file update always requires to work // on latest file status. - segmentDetails = - SegmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath()); + segmentDetails = SegmentStatusManager.readLoadMetadata( + CarbonTablePath.getMetadataPath(identifier.getTablePath())); if (segmentDetails.length > 0) { isPartitionTable = segmentDetails[0].getSegmentFile() != null; } @@ -147,8 +142,6 @@ public class SegmentUpdateStatusManager { } - - /** * Returns the LoadMetadata Details * @return @@ -179,7 +172,7 @@ public class SegmentUpdateStatusManager { * @return */ public ICarbonLock getTableUpdateStatusLock() { - return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, + return CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_UPDATE_STATUS_LOCK); } @@ -195,7 +188,8 @@ public class SegmentUpdateStatusManager { new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE); String endTimeStamp = ""; String startTimeStamp = ""; - String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId); + String segmentPath = CarbonTablePath.getSegmentPath( + identifier.getTablePath(), segmentId); CarbonFile segDir = FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath)); for (LoadMetadataDetails eachSeg : segmentDetails) { @@ -253,7 +247,7 @@ public class SegmentUpdateStatusManager { * @throws Exception */ public String[] getDeleteDeltaFilePath(String blockFilePath, String segmentId) throws Exception { - String blockId = CarbonUtil.getBlockId(absoluteTableIdentifier, blockFilePath, segmentId); + String blockId = CarbonUtil.getBlockId(identifier, blockFilePath, segmentId); String tupleId; if (isPartitionTable) { tupleId = CarbonTablePath.getShortBlockIdForPartitionTable(blockId); @@ -274,20 +268,18 @@ public class SegmentUpdateStatusManager { */ public List getDeltaFiles(String tupleId, String extension) throws Exception { try { - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); String segment = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID); String completeBlockName = CarbonTablePath.addDataPartPrefix( CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCK_ID) + CarbonCommonConstants.FACT_FILE_EXT); String blockPath; if (isPartitionTable) { - blockPath = absoluteTableIdentifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + blockPath = identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.PART_ID) .replace("#", "/") + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName; } else { - String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segment); + String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath( + identifier.getTablePath(), segment); blockPath = carbonDataDirectoryPath + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName; } @@ -411,12 +403,8 @@ public class SegmentUpdateStatusManager { * @return */ public CarbonFile[] getDeleteDeltaFilesList(final Segment segmentId, final String blockName) { - - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - - String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId.getSegmentNo()); + String segmentPath = CarbonTablePath.getSegmentPath( + identifier.getTablePath(), segmentId.getSegmentNo()); CarbonFile segDir = FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath)); @@ -463,16 +451,12 @@ public class SegmentUpdateStatusManager { final String fileExtension, final boolean excludeOriginalFact, CarbonFile[] allFilesOfSegment, boolean isAbortedFile) { - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier); String endTimeStamp = ""; String startTimeStamp = ""; long factTimeStamp = 0; - LoadMetadataDetails[] segmentDetails = - segmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath()); + LoadMetadataDetails[] segmentDetails = SegmentStatusManager.readLoadMetadata( + CarbonTablePath.getMetadataPath(identifier.getTablePath())); for (LoadMetadataDetails eachSeg : segmentDetails) { if (eachSeg.getLoadName().equalsIgnoreCase(segmentId)) { @@ -677,13 +661,9 @@ public class SegmentUpdateStatusManager { return new SegmentUpdateDetails[0]; } - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - String tableUpdateStatusPath = - carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR - + tableUpdateStatusIdentifier; + CarbonTablePath.getMetadataPath(identifier.getTablePath()) + + CarbonCommonConstants.FILE_SEPARATOR + tableUpdateStatusIdentifier; AtomicFileOperations fileOperation = new AtomicFileOperationsImpl(tableUpdateStatusPath, FileFactory.getFileType(tableUpdateStatusPath)); @@ -725,13 +705,9 @@ public class SegmentUpdateStatusManager { */ public void writeLoadDetailsIntoFile(List listOfSegmentUpdateDetailsArray, String updateStatusFileIdentifier) throws IOException { - - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); - String fileLocation = - carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR + CarbonTablePath.getMetadataPath(identifier.getTablePath()) + + CarbonCommonConstants.FILE_SEPARATOR + CarbonUpdateUtil.getUpdateStatusFileName(updateStatusFileIdentifier); AtomicFileOperations fileWrite = http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java ---------------------------------------------------------------------- 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 0cc783e..ec82630 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 @@ -17,29 +17,12 @@ package org.apache.carbondata.core.util; -import java.io.BufferedReader; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.Closeable; -import java.io.DataInputStream; -import java.io.File; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.ObjectInputStream; -import java.io.UnsupportedEncodingException; +import java.io.*; import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Set; +import java.util.*; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; @@ -89,7 +72,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager; import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager; import org.apache.carbondata.core.util.comparator.Comparator; import org.apache.carbondata.core.util.comparator.SerializableComparator; -import org.apache.carbondata.core.util.path.CarbonStorePath; import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.format.BlockletHeader; import org.apache.carbondata.format.DataChunk2; @@ -1088,20 +1070,18 @@ public final class CarbonUtil { * * @param taskId * @param tableBlockInfoList - * @param absoluteTableIdentifier + * @param identifier */ public static long calculateDriverBTreeSize(String taskId, String bucketNumber, - List tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier) { + List tableBlockInfoList, AbsoluteTableIdentifier identifier) { // need to sort the block info list based for task in ascending order so // it will be sinkup with block index read from file Collections.sort(tableBlockInfoList); - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); // geting the index file path //TODO need to pass proper partition number when partiton will be supported - String carbonIndexFilePath = carbonTablePath - .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(), + String carbonIndexFilePath = CarbonTablePath + .getCarbonIndexFilePath(identifier.getTablePath(), taskId, + tableBlockInfoList.get(0).getSegmentId(), bucketNumber, CarbonTablePath.DataFileUtil .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()), tableBlockInfoList.get(0).getVersion()); @@ -1328,23 +1308,21 @@ public final class CarbonUtil { * * @param taskId task id of the file * @param tableBlockInfoList list of table block - * @param absoluteTableIdentifier absolute table identifier + * @param identifier absolute table identifier * @return list of block info * @throws IOException if any problem while reading */ public static List readCarbonIndexFile(String taskId, String bucketNumber, - List tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier) + List tableBlockInfoList, AbsoluteTableIdentifier identifier) throws IOException { // need to sort the block info list based for task in ascending order so // it will be sinkup with block index read from file Collections.sort(tableBlockInfoList); - CarbonTablePath carbonTablePath = CarbonStorePath - .getCarbonTablePath(absoluteTableIdentifier.getTablePath(), - absoluteTableIdentifier.getCarbonTableIdentifier()); // geting the index file path //TODO need to pass proper partition number when partiton will be supported - String carbonIndexFilePath = carbonTablePath - .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(), + String carbonIndexFilePath = CarbonTablePath + .getCarbonIndexFilePath(identifier.getTablePath(), taskId, + tableBlockInfoList.get(0).getSegmentId(), bucketNumber, CarbonTablePath.DataFileUtil .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()), tableBlockInfoList.get(0).getVersion()); @@ -2204,21 +2182,6 @@ public final class CarbonUtil { } /** - * get the parent folder of old table path and returns the new tablePath by appending new - * tableName to the parent - * - * @param carbonTablePath Old tablePath - * @param newTableName new table name - * @return the new table path - */ - public static String getNewTablePath( - Path carbonTablePath, - String newTableName) { - Path parentPath = carbonTablePath.getParent(); - return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + newTableName; - } - - /** * This method will calculate the data size and index size for carbon table */ public static Map calculateDataIndexSize(CarbonTable carbonTable) @@ -2228,18 +2191,17 @@ public final class CarbonUtil { long indexSize = 0L; long lastUpdateTime = 0L; boolean needUpdate = false; - AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier(); - CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier); + AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier(); String isCalculated = CarbonProperties.getInstance() .getProperty(CarbonCommonConstants.ENABLE_CALCULATE_SIZE, CarbonCommonConstants.DEFAULT_ENABLE_CALCULATE_SIZE); if (isCalculated.equalsIgnoreCase("true")) { - SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier); + SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier); ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock(); try { if (carbonLock.lockWithRetries()) { LOGGER.info("Acquired lock for table for table status updation"); - String metadataPath = carbonTable.getMetaDataFilepath(); + String metadataPath = carbonTable.getMetadataPath(); LoadMetadataDetails[] loadMetadataDetails = SegmentStatusManager.readLoadMetadata(metadataPath); @@ -2253,8 +2215,8 @@ public final class CarbonUtil { if (null == dsize || null == isize) { needUpdate = true; LOGGER.info("It is an old segment, need calculate data size and index size again"); - HashMap map = CarbonUtil - .getDataSizeAndIndexSize(carbonTablePath, loadMetadataDetail.getLoadName()); + HashMap map = CarbonUtil.getDataSizeAndIndexSize( + identifier.getTablePath(), loadMetadataDetail.getLoadName()); dsize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE)); isize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE)); loadMetadataDetail.setDataSize(dsize); @@ -2266,10 +2228,12 @@ public final class CarbonUtil { } // If it contains old segment, write new load details if (needUpdate) { - SegmentStatusManager.writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(), + SegmentStatusManager.writeLoadDetailsIntoFile( + CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()), loadMetadataDetails); } - String tableStatusPath = carbonTablePath.getTableStatusFilePath(); + String tableStatusPath = + CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()); if (FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) { lastUpdateTime = FileFactory.getCarbonFile(tableStatusPath, FileFactory.getFileType(tableStatusPath)) @@ -2296,12 +2260,12 @@ public final class CarbonUtil { } // Get the total size of carbon data and the total size of carbon index - private static HashMap getDataSizeAndIndexSize(CarbonTablePath carbonTablePath, + private static HashMap getDataSizeAndIndexSize(String tablePath, String segmentId) throws IOException { long carbonDataSize = 0L; long carbonIndexSize = 0L; HashMap dataAndIndexSize = new HashMap(); - String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId); + String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId); FileFactory.FileType fileType = FileFactory.getFileType(segmentPath); switch (fileType) { case HDFS: @@ -2370,21 +2334,19 @@ public final class CarbonUtil { } // Get the total size of carbon data and the total size of carbon index - public static HashMap getDataSizeAndIndexSize(CarbonTablePath carbonTablePath, + public static HashMap getDataSizeAndIndexSize(String tablePath, Segment segment) throws IOException { if (segment.getSegmentFileName() != null) { - SegmentFileStore fileStore = - new SegmentFileStore(carbonTablePath.getPath(), segment.getSegmentFileName()); + SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName()); return getDataSizeAndIndexSize(fileStore); } else { - return getDataSizeAndIndexSize(carbonTablePath, segment.getSegmentNo()); + return getDataSizeAndIndexSize(tablePath, segment.getSegmentNo()); } } // Get the total size of segment. - public static long getSizeOfSegment(CarbonTablePath carbonTablePath, - Segment segment) throws IOException { - HashMap dataSizeAndIndexSize = getDataSizeAndIndexSize(carbonTablePath, segment); + public static long getSizeOfSegment(String tablePath, Segment segment) throws IOException { + HashMap dataSizeAndIndexSize = getDataSizeAndIndexSize(tablePath, segment); long size = 0; for (Long eachSize: dataSizeAndIndexSize.values()) { size += eachSize; @@ -2506,9 +2468,7 @@ public final class CarbonUtil { String blockName = filePath.substring(filePath.lastIndexOf("/") + 1, filePath.length()); String tablePath = identifier.getTablePath(); if (filePath.startsWith(tablePath)) { - String factDir = - CarbonStorePath.getCarbonTablePath(tablePath, identifier.getCarbonTableIdentifier()) - .getFactDir(); + String factDir = CarbonTablePath.getFactDir(tablePath); if (filePath.startsWith(factDir)) { blockId = "Part0" + CarbonCommonConstants.FILE_SEPARATOR + "Segment_" + segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName; http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java deleted file mode 100644 index b864e6e..0000000 --- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java +++ /dev/null @@ -1,71 +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.util.path; - -import java.io.File; - -/** - * Helps to get Shared dimension files path. - */ -public class CarbonSharedDictionaryPath { - - private static final String SHAREDDIM_DIR = "SharedDictionary"; - private static final String DICTIONARY_EXT = ".dict"; - private static final String DICTIONARY_META_EXT = ".dictmeta"; - private static final String SORT_INDEX_EXT = ".sortindex"; - - /*** - * @param storePath store path - * @param databaseName data base name - * @param columnId unique column identifier - * @return absolute path of shared dictionary file - */ - public static String getDictionaryFilePath(String storePath, String databaseName, - String columnId) { - return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId - + DICTIONARY_EXT; - } - - /*** - * @param storePath store path - * @param databaseName data base name - * @param columnId unique column identifier - * @return absolute path of shared dictionary meta file - */ - public static String getDictionaryMetaFilePath(String storePath, String databaseName, - String columnId) { - return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId - + DICTIONARY_META_EXT; - } - - /*** - * @param storePath store path - * @param databaseName data base name - * @param columnId unique column identifier - * @return absolute path of shared dictionary sort index file - */ - public static String getSortIndexFilePath(String storePath, String databaseName, - String columnId) { - return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId - + SORT_INDEX_EXT; - } - - private static String getSharedDictionaryDir(String storePath, String databaseName) { - return storePath + File.separator + databaseName + File.separator + SHAREDDIM_DIR; - } - -} http://git-wip-us.apache.org/repos/asf/carbondata/blob/9b9125b6/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java deleted file mode 100644 index e57448d..0000000 --- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java +++ /dev/null @@ -1,71 +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.util.path; - -import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; -import org.apache.carbondata.core.metadata.CarbonTableIdentifier; - -import org.apache.hadoop.fs.Path; - -/** - * Helps to get Store content paths. - */ -public class CarbonStorePath extends Path { - - private String storePath; - - public CarbonStorePath(String storePathString) { - super(storePathString); - this.storePath = storePathString; - } - - /** - * gets CarbonTablePath object to manage table paths - * - * @param tablePath the store path of the segment - * @param tableIdentifier identifier of carbon table that the segment belong to - * @return the store location of the segment - */ - public static CarbonTablePath getCarbonTablePath(String tablePath, - CarbonTableIdentifier tableIdentifier) { - return new CarbonTablePath(tableIdentifier, tablePath); - } - - public static CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier identifier) { - CarbonTableIdentifier id = identifier.getCarbonTableIdentifier(); - return new CarbonTablePath(id, identifier.getTablePath()); - } - - /** - * gets CarbonTablePath object to manage table paths - */ - public CarbonTablePath getCarbonTablePath(CarbonTableIdentifier tableIdentifier) { - return CarbonStorePath.getCarbonTablePath(storePath, tableIdentifier); - } - - @Override public boolean equals(Object o) { - if (!(o instanceof CarbonStorePath)) { - return false; - } - CarbonStorePath path = (CarbonStorePath)o; - return storePath.equals(path.storePath) && super.equals(o); - } - - @Override public int hashCode() { - return super.hashCode() + storePath.hashCode(); - } -}