You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/07/20 10:14:09 UTC

[41/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
index 2d18b54,0000000..d0b0453
mode 100644,000000..100644
--- a/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
+++ b/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
@@@ -1,171 -1,0 +1,168 @@@
 +/*
 + * 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.carbondata.lcm.locks;
 +
 +import java.util.Collections;
 +import java.util.List;
 +
 +import org.carbondata.common.logging.LogService;
 +import org.carbondata.common.logging.LogServiceFactory;
++import org.carbondata.core.carbon.CarbonTableIdentifier;
 +import org.carbondata.core.constants.CarbonCommonConstants;
 +
 +import org.apache.zookeeper.CreateMode;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.ZooDefs.Ids;
 +import org.apache.zookeeper.ZooKeeper;
 +
 +/**
 + * For Handling the zookeeper locking implementation
 + */
 +public class ZooKeeperLocking extends AbstractCarbonLock {
 +
 +  private static final LogService LOGGER =
 +      LogServiceFactory.getLogService(ZooKeeperLocking.class.getName());
 +
 +  /**
 +   * zk is the zookeeper client instance
 +   */
 +  private static ZooKeeper zk;
 +
 +  /**
 +   * zooKeeperLocation is the location in the zoo keeper file system where the locks will be
 +   * maintained.
 +   */
 +  private static final String zooKeeperLocation = CarbonCommonConstants.ZOOKEEPER_LOCATION;
 +
 +  /**
 +   * Unique folder for each table with DatabaseName_TableName
 +   */
 +  private final String tableIdFolder;
 +
 +  /**
 +   * lockName is the name of the lock to use. This name should be same for every process that want
 +   * to share the same lock
 +   */
 +  private String lockName;
 +
 +  /**
 +   * lockPath is the unique path created for the each instance of the carbon lock.
 +   */
 +  private String lockPath;
 +
 +  private String lockTypeFolder;
 +
 +  /**
-    * @param lockUsage
++   * @param tableIdentifier
++   * @param lockFile
 +   */
-   public ZooKeeperLocking(String location, LockUsage lockUsage) {
-     this.lockName = CarbonCommonConstants.ZOOKEEPER_LOCK;
-     this.lockTypeFolder = zooKeeperLocation;
-     location = location.replace("\\", "/");
-     String tempStr = location.substring(0, location.lastIndexOf('/'));
-     String databaseName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
- 
-     String tableName = location.substring(location.lastIndexOf('/') + 1, location.length());
- 
-     this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + databaseName
-         + '.' + tableName;
++  public ZooKeeperLocking(CarbonTableIdentifier tableIdentifier, String lockFile) {
++    this.lockName = lockFile;
++    this.tableIdFolder =
++        zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + tableIdentifier.getDatabaseName()
++            + '.' + tableIdentifier.getTableName();
 +
 +    zk = ZookeeperInit.getInstance().getZookeeper();
 +
-     this.lockTypeFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + databaseName
-         + '.' + tableName + CarbonCommonConstants.FILE_SEPARATOR
-         + lockUsage.toString();
++    this.lockTypeFolder =
++        zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + tableIdentifier.getDatabaseName()
++            + '.' + tableIdentifier.getTableName() + CarbonCommonConstants.FILE_SEPARATOR
++            + lockFile;
 +    try {
 +      createBaseNode();
 +      // if exists returns null then path doesnt exist. so creating.
 +      if (null == zk.exists(this.tableIdFolder, true)) {
 +        zk.create(this.tableIdFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 +      }
 +      // if exists returns null then path doesnt exist. so creating.
 +      if (null == zk.exists(this.lockTypeFolder, true)) {
 +        zk.create(this.lockTypeFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 +      }
 +    } catch (KeeperException | InterruptedException e) {
 +      LOGGER.error(e, e.getMessage());
 +    }
 +    initRetry();
 +  }
 +
 +  /**
 +   * Creating a znode in which all the znodes (lock files )are maintained.
 +   */
 +  private void createBaseNode() throws KeeperException, InterruptedException {
 +    if (null == zk.exists(zooKeeperLocation, true)) {
 +      // creating a znode in which all the znodes (lock files )are maintained.
 +      zk.create(zooKeeperLocation, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
 +    }
 +  }
 +
 +  /**
 +   * Handling of the locking mechanism using zoo keeper.
 +   */
 +  @Override public boolean lock() {
 +    try {
 +      // create the lock file with lockName.
 +      lockPath =
 +          zk.create(this.lockTypeFolder + CarbonCommonConstants.FILE_SEPARATOR + lockName, null,
 +              Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
 +
 +      // get the children present in zooKeeperLocation.
 +      List<String> nodes = zk.getChildren(this.lockTypeFolder, null);
 +
 +      // sort the childrens
 +      Collections.sort(nodes);
 +
 +      // here the logic is , for each lock request zookeeper will create a file ending with
 +      // incremental digits.
 +      // so first request will be 00001 next is 00002 and so on.
 +      // if the current request is 00002 and already one previous request(00001) is present then get
 +      // children will give both nodes.
 +      // after the sort we are checking if the lock path is first or not .if it is first then lock
 +      // has been acquired.
 +
 +      if (lockPath.endsWith(nodes.get(0))) {
 +        return true;
 +      } else {
 +        // if locking failed then deleting the created lock as next time again new lock file will be
 +        // created.
 +        zk.delete(lockPath, -1);
 +        return false;
 +      }
 +    } catch (KeeperException | InterruptedException e) {
 +      LOGGER.error(e, e.getMessage());
 +      return false;
 +    }
 +  }
 +
 +  /**
 +   * @return status where lock file is unlocked or not.
 +   */
 +  @Override public boolean unlock() {
 +    try {
 +      // exists will return null if the path doesn't exists.
 +      if (null != zk.exists(lockPath, true)) {
 +        zk.delete(lockPath, -1);
 +        lockPath = null;
 +      }
 +    } catch (KeeperException | InterruptedException e) {
 +      LOGGER.error(e, e.getMessage());
 +      return false;
 +    }
 +    return true;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
index 3949f47,e6220d8..f93d25e
--- a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
@@@ -228,12 -240,13 +240,13 @@@ public class SegmentStatusManager 
    /**
     * updates deletion status
     * @param loadIds
 -   * @param cubeFolderPath
 +   * @param tableFolderPath
     * @return
     */
 -  public List<String> updateDeletionStatus(List<String> loadIds, String cubeFolderPath) {
 +  public List<String> updateDeletionStatus(List<String> loadIds, String tableFolderPath) {
-     ICarbonLock carbonLock =
-         CarbonLockFactory.getCarbonLockObj(tableFolderPath, LockUsage.METADATA_LOCK);
+     ICarbonLock carbonLock = CarbonLockFactory
+         .getCarbonLockObj(absoluteTableIdentifier.getCarbonTableIdentifier(),
+             LockUsage.METADATA_LOCK);
      List<String> invalidLoadIds = new ArrayList<String>(0);
      try {
        if (carbonLock.lockWithRetries()) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/api/dataloader/SchemaInfo.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/processing/api/dataloader/SchemaInfo.java
index c3d71ec,c52e3a0..2582260
--- a/processing/src/main/java/org/carbondata/processing/api/dataloader/SchemaInfo.java
+++ b/processing/src/main/java/org/carbondata/processing/api/dataloader/SchemaInfo.java
@@@ -162,10 -166,26 +166,26 @@@ public class SchemaInfo 
    }
  
    /**
 -   * @param schemaName the schemaName to set
 +   * @param databaseName the databaseName to set
     */
 -  public void setSchemaName(String schemaName) {
 -    this.schemaName = schemaName;
 +  public void setDatabaseName(String databaseName) {
 +    this.databaseName = databaseName;
    }
  
+   /**
+    * the method returns the value to be treated as null while data load
+    * @return
+    */
+   public String getSerializationNullFormat() {
+     return serializationNullFormat;
+   }
+ 
+   /**
+    * the method sets the value to be treated as null while data load
+    * @param serializationNullFormat
+    */
+   public void setSerializationNullFormat(String serializationNullFormat) {
+     this.serializationNullFormat = serializationNullFormat;
+   }
+ 
  }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/csvload/GraphExecutionUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
index 2fc0454,3b69b4a..21ac146
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
@@@ -368,9 -366,11 +368,11 @@@ public class CsvInput extends BaseStep 
          }
        }
      }
 -    LOGGER.info("*****************Started ALL ALL csv reading***********");
 +    LOGGER.info("*****************Started all csv reading***********");
      startProcess(numberOfNodes);
 -    LOGGER.info("*****************Completed ALL ALL csv reading***********");
 +    LOGGER.info("*****************Completed all csv reading***********");
+     CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordCsvInputStepTime(
+         meta.getPartitionID(), System.currentTimeMillis());
      setOutputDone();
      return false;
    }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
index 619da5e,8ddb429..6f7a2b1
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@@ -39,15 -39,12 +39,12 @@@ import org.carbondata.core.carbon.metad
  import org.carbondata.core.carbon.path.CarbonStorePath;
  import org.carbondata.core.carbon.path.CarbonTablePath;
  import org.carbondata.core.constants.CarbonCommonConstants;
 -import org.carbondata.core.file.manager.composite.FileData;
 -import org.carbondata.core.file.manager.composite.FileManager;
 -import org.carbondata.core.file.manager.composite.IFileManagerComposite;
  import org.carbondata.core.keygenerator.KeyGenException;
- import org.carbondata.core.util.CarbonProperties;
- import org.carbondata.core.util.CarbonUtil;
- import org.carbondata.core.util.CarbonUtilException;
- import org.carbondata.core.util.DataTypeUtil;
+ import org.carbondata.core.util.*;
  import org.carbondata.processing.datatypes.GenericDataType;
 +import org.carbondata.processing.mdkeygen.file.FileData;
 +import org.carbondata.processing.mdkeygen.file.FileManager;
 +import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
  import org.carbondata.processing.store.CarbonDataFileAttributes;
  import org.carbondata.processing.store.CarbonFactDataHandlerColumnar;
  import org.carbondata.processing.store.CarbonFactDataHandlerModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
index 6ef58fc,dac4b6c..b20bcf0
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
@@@ -34,11 -34,14 +34,14 @@@ import org.carbondata.common.logging.Lo
  import org.carbondata.common.logging.LogServiceFactory;
  import org.carbondata.core.cache.dictionary.Dictionary;
  import org.carbondata.core.constants.CarbonCommonConstants;
 -import org.carbondata.core.file.manager.composite.IFileManagerComposite;
  import org.carbondata.core.keygenerator.KeyGenException;
+ import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+ import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
  import org.carbondata.core.writer.HierarchyValueWriterForCSV;
  import org.carbondata.processing.datatypes.GenericDataType;
 +import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
  import org.carbondata.processing.schema.metadata.ArrayWrapper;
+ import org.carbondata.processing.schema.metadata.ColumnSchemaDetails;
  import org.carbondata.processing.schema.metadata.ColumnsInfo;
  
  import it.unimi.dsi.fastutil.ints.Int2ObjectMap;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --cc processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
index e41ed99,0000000..c4f5093
mode 100644,000000..100644
--- a/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
+++ b/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
@@@ -1,48 -1,0 +1,51 @@@
 +/**
 + *
 + */
 +package org.carbondata.lcm.locks;
 +
 +import java.io.File;
 +
++import org.carbondata.core.carbon.AbsoluteTableIdentifier;
++import org.carbondata.core.carbon.CarbonTableIdentifier;
 +import org.junit.After;
 +import org.junit.Before;
 +import org.junit.Test;
 +import org.pentaho.di.core.util.Assert;
 +
 +/**
 + * Test class to test the functionality of the local file locking.
 + *
 + * @author Administrator
 + */
 +public class LocalFileLockTest {
 +
 +  /**
 +   * @throws java.lang.Exception
 +   */
 +  @Before public void setUp() throws Exception {
 +  }
 +
 +  /**
 +   * @throws java.lang.Exception
 +   */
 +  @After public void tearDown() throws Exception {
 +  }
 +
 +  @Test public void testingLocalFileLockingByAcquiring2Locks() {
 +
++	CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("databaseName", "tableName", "tableId");
 +    LocalFileLock localLock1 =
-         new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
++        new LocalFileLock(carbonTableIdentifier,
 +            LockUsage.METADATA_LOCK);
 +    Assert.assertTrue(localLock1.lock());
 +    LocalFileLock localLock2 =
-         new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
++        new LocalFileLock(carbonTableIdentifier,
 +            LockUsage.METADATA_LOCK);
 +    Assert.assertTrue(!localLock2.lock());
 +
 +    Assert.assertTrue(localLock1.unlock());
 +    Assert.assertTrue(localLock2.lock());
 +
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eaecb651/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --cc processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
index 87616e7,0000000..2a5644f
mode 100644,000000..100644
--- a/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
+++ b/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
@@@ -1,125 -1,0 +1,128 @@@
 +/**
 + *
 + */
 +package org.carbondata.lcm.locks;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.net.ServerSocket;
 +import java.util.Properties;
 +
++import org.carbondata.core.carbon.CarbonTableIdentifier;
 +import org.carbondata.core.util.CarbonProperties;
 +
 +import mockit.NonStrictExpectations;
++
 +import org.apache.zookeeper.server.ServerConfig;
 +import org.apache.zookeeper.server.ZooKeeperServerMain;
 +import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +/**
 + * @author Administrator
 + */
 +public class ZooKeeperLockingTest {
 +
 +  int freePort;
 +
 +  /**
 +   * @throws java.lang.Exception
 +   */
 +  @Before public void setUp() throws Exception {
 +    Properties startupProperties = new Properties();
 +    startupProperties.setProperty("dataDir", (new File("./target").getAbsolutePath()));
 +    startupProperties.setProperty("dataLogDir", (new File("./target").getAbsolutePath()));
 +    freePort = findFreePort();
 +    startupProperties.setProperty("clientPort", "" + freePort);
 +    QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
 +    try {
 +      quorumConfiguration.parseProperties(startupProperties);
 +    } catch (Exception e) {
 +      throw new RuntimeException(e);
 +    }
 +
 +    final ZooKeeperServerMain zooKeeperServer = new ZooKeeperServerMain();
 +    final ServerConfig configuration = new ServerConfig();
 +    configuration.readFrom(quorumConfiguration);
 +    new Thread() {
 +      public void run() {
 +        try {
 +          zooKeeperServer.runFromConfig(configuration);
 +        } catch (IOException e) {
 +          System.out.println("ZooKeeper failure");
 +        }
 +      }
 +    }.start();
 +  }
 +
 +  /**
 +   * @throws java.lang.Exception
 +   */
 +  @After public void tearDown() throws Exception {
 +  }
 +
 +  @Test public void testZooKeeperLockingByTryingToAcquire2Locks()
 +      throws IllegalArgumentException, IllegalAccessException, NoSuchFieldException,
 +      SecurityException {
 +
 +    final CarbonProperties cp = CarbonProperties.getInstance();
 +    new NonStrictExpectations(cp) {
 +      {
 +        cp.getProperty("/CarbonLocks");
 +        result = "/carbontests";
 +        cp.getProperty("spark.deploy.zookeeper.url");
 +        result = "127.0.0.1:" + freePort;
 +      }
 +    };
 +
 +    ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
 +
++    CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("dbName", "tableName", "tableId");
 +    ZooKeeperLocking zkl =
-         new ZooKeeperLocking("D:/carbondata/examples/target/store/default/t3/Metadata",
++        new ZooKeeperLocking(tableIdentifier,
 +            LockUsage.METADATA_LOCK);
 +    Assert.assertTrue(zkl.lock());
 +
 +    ZooKeeperLocking zk2 = new ZooKeeperLocking(
-         "D:/carbondata/examples/target/store/default/t3/Metadata", LockUsage.METADATA_LOCK);
++    		tableIdentifier, LockUsage.METADATA_LOCK);
 +    Assert.assertTrue(!zk2.lock());
 +
 +    Assert.assertTrue(zkl.unlock());
 +    Assert.assertTrue(zk2.lock());
 +    Assert.assertTrue(zk2.unlock());
 +  }
 +
 +  /**
 +   * For finding the free port available.
 +   *
 +   * @return
 +   */
 +  private static int findFreePort() {
 +    ServerSocket socket = null;
 +    try {
 +      socket = new ServerSocket(0);
 +      socket.setReuseAddress(true);
 +      int port = socket.getLocalPort();
 +      try {
 +        socket.close();
 +      } catch (IOException e) {
 +        // Ignore IOException on close()
 +      }
 +      return port;
 +    } catch (Exception e) {
 +      // Ignore
 +    } finally {
 +      if (socket != null) {
 +        try {
 +          socket.close();
 +        } catch (IOException e) {
 +          throw new RuntimeException(e);
 +        }
 +      }
 +    }
 +    return 2181;
 +  }
 +}