You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@carbondata.apache.org by ManoharVanam <gi...@git.apache.org> on 2016/09/14 17:22:28 UTC

[GitHub] incubator-carbondata pull request #156: [WIP] Added table Status lock while ...

GitHub user ManoharVanam opened a pull request:

    https://github.com/apache/incubator-carbondata/pull/156

    [WIP] Added table Status lock while deleting the segments

    Added table Status lock while deleting the segments
    throwing exception if lock is not available

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ManoharVanam/incubator-carbondata DeleteSEg

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-carbondata/pull/156.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #156
    
----
commit d6d5304b6e4743f1c081232d449b20691a6ca0e1
Author: Manohar <ma...@gmail.com>
Date:   2016-09-14T17:18:23Z

    Added table Status lock while deleting the segments

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79291307
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/locks/CarbonLockUtil.java ---
    @@ -0,0 +1,64 @@
    +/*
    + * 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.lcm.locks;
    +
    +import org.apache.carbondata.common.logging.LogService;
    +import org.apache.carbondata.common.logging.LogServiceFactory;
    +
    +/**
    + * This is a singleton class for initialization of zookeeper client.
    --- End diff --
    
    provide proper comment


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79187047
  
    --- Diff: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala ---
    @@ -1077,19 +1077,38 @@ object CarbonDataRDDFactory extends Logging {
           val segmentStatusManager = new SegmentStatusManager(table.getAbsoluteTableIdentifier)
           val details = segmentStatusManager
             .readLoadMetadata(loadMetadataFilePath)
    +      val carbonTableStatusLock = CarbonLockFactory
    +        .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +          LockUsage.TABLE_STATUS_LOCK)
     
           // Delete marked loads
           val isUpdationRequired = DeleteLoadFolders
             .deleteLoadFoldersFromFileSystem(carbonLoadModel, hdfsStoreLocation,
               partitioner.partitionCount, isForceDeletion, details)
     
           if (isUpdationRequired) {
    +        try {
             // Update load metadate file after cleaning deleted nodes
    -        CarbonLoaderUtil.writeLoadMetadata(
    -          carbonLoadModel.getCarbonDataLoadSchema,
    -          carbonLoadModel.getDatabaseName,
    -          carbonLoadModel.getTableName, details.toList.asJava
    -        )
    +        if (carbonTableStatusLock.lockWithRetries()) {
    +          logger.info("Table status lock has been successfully acquired.")
    +          CarbonLoaderUtil.writeLoadMetadata(
    +            carbonLoadModel.getCarbonDataLoadSchema,
    +            carbonLoadModel.getDatabaseName,
    +            carbonLoadModel.getTableName, details.toList.asJava
    +          )
    +        }
    +        else {
    +          val errorMsg = "Clean files request is failed for " + carbonLoadModel.getDatabaseName +
    +                         "." + carbonLoadModel.getTableName +
    +                         ". Not able to acquire the table status lock."
    --- End diff --
    
    also mention that "due to  other operation running in the  background ", this message will give more clear picture to user about the task failure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [WIP] Added table Status lock while ...

Posted by ManoharVanam <gi...@git.apache.org>.
Github user ManoharVanam commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79119024
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -246,15 +247,22 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * updates deletion status
        * @param loadIds
        * @param tableFolderPath
    +   * @param dbName
    +   * @param tableName
        * @return
        */
    -  public List<String> updateDeletionStatus(List<String> loadIds, String tableFolderPath) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    +  public List<String> updateDeletionStatus(List<String> loadIds, String tableFolderPath,
    +      String dbName, String tableName) throws Exception {
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by akashrn5 <gi...@git.apache.org>.
Github user akashrn5 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79291255
  
    --- Diff: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala ---
    @@ -1111,25 +1131,32 @@ object CarbonDataRDDFactory extends Logging {
         val table = org.apache.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
           .getCarbonTable(carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName)
         val metaDataPath: String = table.getMetaDataFilepath
    -    val carbonLock = CarbonLockFactory
    +    val carbonCleanFilesLock = CarbonLockFactory
           .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    -        LockUsage.METADATA_LOCK
    +        LockUsage.CLEAN_FILES_LOCK
           )
         try {
    -      if (carbonLock.lockWithRetries()) {
    +      if (carbonCleanFilesLock.lockWithRetries()) {
    +        logger.info("Clean files lock has been successfully acquired.")
             deleteLoadsAndUpdateMetadata(carbonLoadModel,
               table,
               partitioner,
               hdfsStoreLocation,
               isForceDeletion = true)
           }
    +      else {
    +        val errorMsg = "Clean files request is failed for " + carbonLoadModel.getDatabaseName +
    --- End diff --
    
     "Clean file request is running in the background, please try after some time". u[date the comment since already we know the operation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [WIP] Added table Status lock while ...

Posted by manishgupta88 <gi...@git.apache.org>.
Github user manishgupta88 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79111960
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -306,13 +330,17 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * @return
        */
       public List<String> updateDeletionStatus(String loadDate, String tableFolderPath,
    -      Long loadStartTime) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    +      Long loadStartTime, String dbName, String tableName) throws Exception {
    --- End diff --
    
    same comment as above...take the database and table name from carbon table identifier instance


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by ManoharVanam <gi...@git.apache.org>.
Github user ManoharVanam commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79217913
  
    --- Diff: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala ---
    @@ -1077,19 +1077,38 @@ object CarbonDataRDDFactory extends Logging {
           val segmentStatusManager = new SegmentStatusManager(table.getAbsoluteTableIdentifier)
           val details = segmentStatusManager
             .readLoadMetadata(loadMetadataFilePath)
    +      val carbonTableStatusLock = CarbonLockFactory
    +        .getCarbonLockObj(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
    +          LockUsage.TABLE_STATUS_LOCK)
     
           // Delete marked loads
           val isUpdationRequired = DeleteLoadFolders
             .deleteLoadFoldersFromFileSystem(carbonLoadModel, hdfsStoreLocation,
               partitioner.partitionCount, isForceDeletion, details)
     
           if (isUpdationRequired) {
    +        try {
             // Update load metadate file after cleaning deleted nodes
    -        CarbonLoaderUtil.writeLoadMetadata(
    -          carbonLoadModel.getCarbonDataLoadSchema,
    -          carbonLoadModel.getDatabaseName,
    -          carbonLoadModel.getTableName, details.toList.asJava
    -        )
    +        if (carbonTableStatusLock.lockWithRetries()) {
    +          logger.info("Table status lock has been successfully acquired.")
    +          CarbonLoaderUtil.writeLoadMetadata(
    +            carbonLoadModel.getCarbonDataLoadSchema,
    +            carbonLoadModel.getDatabaseName,
    +            carbonLoadModel.getTableName, details.toList.asJava
    +          )
    +        }
    +        else {
    +          val errorMsg = "Clean files request is failed for " + carbonLoadModel.getDatabaseName +
    +                         "." + carbonLoadModel.getTableName +
    +                         ". Not able to acquire the table status lock."
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79291346
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/locks/LockUsage.java ---
    @@ -28,5 +28,7 @@
       public static String COMPACTION_LOCK = "compaction.lock";
       public static String SYSTEMLEVEL_COMPACTION_LOCK = "system_level_compaction.lock";
       public static String TABLE_STATUS_LOCK = "tablestatus.lock";
    +  public static String DELETE_SEGMENT_LOCK = "delete_segment.lock";
    --- End diff --
    
    make it final/use enum


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-carbondata/pull/156


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79204842
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -306,13 +330,18 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * @return
        */
       public List<String> updateDeletionStatus(String loadDate, String tableFolderPath,
    -      Long loadStartTime) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    -        .getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier(),
    -            LockUsage.METADATA_LOCK);
    +      Long loadStartTime) throws Exception {
    +    CarbonTableIdentifier carbonTableIdentifier =
    +        absoluteTableIdentifier.getCarbonTableIdentifier();
    +    ICarbonLock carbonMetadataLock =
    +        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.METADATA_LOCK);
    +    ICarbonLock carbonTableStatusLock =
    +        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
    +    String tableDetails =
    +        carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
         List<String> invalidLoadTimestamps = new ArrayList<String>(0);
         try {
    -      if (carbonLock.lockWithRetries()) {
    +      if (carbonMetadataLock.lockWithRetries()) {
    --- End diff --
    
    make it as operational log


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [WIP] Added table Status lock while ...

Posted by manishgupta88 <gi...@git.apache.org>.
Github user manishgupta88 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79111804
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -246,15 +247,22 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * updates deletion status
        * @param loadIds
        * @param tableFolderPath
    +   * @param dbName
    +   * @param tableName
        * @return
        */
    -  public List<String> updateDeletionStatus(List<String> loadIds, String tableFolderPath) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    +  public List<String> updateDeletionStatus(List<String> loadIds, String tableFolderPath,
    +      String dbName, String tableName) throws Exception {
    --- End diff --
    
    I think no need to to change the method signature here...database name and table name can be taken from carbontable identifier and carbonTableIdentifier object is available in the mehod


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [WIP] Added table Status lock while ...

Posted by ManoharVanam <gi...@git.apache.org>.
Github user ManoharVanam commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79119039
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -306,13 +330,17 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * @return
        */
       public List<String> updateDeletionStatus(String loadDate, String tableFolderPath,
    -      Long loadStartTime) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    +      Long loadStartTime, String dbName, String tableName) throws Exception {
    --- End diff --
    
    fixed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by ManoharVanam <gi...@git.apache.org>.
Github user ManoharVanam commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79222066
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -306,13 +330,18 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
        * @return
        */
       public List<String> updateDeletionStatus(String loadDate, String tableFolderPath,
    -      Long loadStartTime) {
    -    ICarbonLock carbonLock = CarbonLockFactory
    -        .getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier(),
    -            LockUsage.METADATA_LOCK);
    +      Long loadStartTime) throws Exception {
    +    CarbonTableIdentifier carbonTableIdentifier =
    +        absoluteTableIdentifier.getCarbonTableIdentifier();
    +    ICarbonLock carbonMetadataLock =
    +        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.METADATA_LOCK);
    +    ICarbonLock carbonTableStatusLock =
    +        CarbonLockFactory.getCarbonLockObj(carbonTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
    +    String tableDetails =
    +        carbonTableIdentifier.getDatabaseName() + "." + carbonTableIdentifier.getTableName();
         List<String> invalidLoadTimestamps = new ArrayList<String>(0);
         try {
    -      if (carbonLock.lockWithRetries()) {
    +      if (carbonMetadataLock.lockWithRetries()) {
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79291710
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -287,12 +307,18 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
             }
     
           } else {
    -        LOG.error("Unable to acquire the metadata lock");
    +        String errorMsg = "Delete segment by id is failed for " + tableDetails
    +            + ". Not able to acquire the delete segment lock due to other operation running "
    --- End diff --
    
    LGTM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-carbondata pull request #156: [CARBONDATA-244] Load and delete seg...

Posted by sujith71955 <gi...@git.apache.org>.
Github user sujith71955 commented on a diff in the pull request:

    https://github.com/apache/incubator-carbondata/pull/156#discussion_r79291381
  
    --- Diff: processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java ---
    @@ -287,12 +307,18 @@ private Integer compareDateValues(Long loadValue, Long userValue) {
             }
     
           } else {
    -        LOG.error("Unable to acquire the metadata lock");
    +        String errorMsg = "Delete segment by id is failed for " + tableDetails
    +            + ". Not able to acquire the delete segment lock due to other operation running "
    --- End diff --
    
    update message, provide operation name


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---