You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/03/02 18:47:27 UTC

[1/2] incubator-carbondata git commit: Added test cases for Dictionary Generators, Dictionary Client and Server to improve the test sifficiency and test code coverage.

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master b93173fcd -> 06f9cf6f3


Added test cases for Dictionary Generators, Dictionary Client and Server to improve the test sifficiency and test code coverage.

Fixed Tests

Fixed Test Failure

Fixed Test Failure

Fixed problem in cleanup affecting other tests.

Fixed tests against latest code.

Fixed Cleanup Issue


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/edaaccb2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/edaaccb2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/edaaccb2

Branch: refs/heads/master
Commit: edaaccb2439ebe6a9ebfa75240aa425ee3c187ee
Parents: b93173f
Author: Vimal-Das <vi...@apache.org>
Authored: Sun Jan 29 20:08:00 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Mar 3 00:16:14 2017 +0530

----------------------------------------------------------------------
 .../dictionary/client/DictionaryClientTest.java | 152 ++++++++++++
 ...ncrementalColumnDictionaryGeneratorTest.java | 201 ++++++++++++++++
 .../ServerDictionaryGeneratorTest.java          | 217 +++++++++++++++++
 .../generator/TableDictionaryGeneratorTest.java | 233 +++++++++++++++++++
 4 files changed, 803 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/edaaccb2/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
new file mode 100644
index 0000000..5d9326b
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/client/DictionaryClientTest.java
@@ -0,0 +1,152 @@
+/*
+ * 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.dictionary.client;
+
+import java.io.File;
+import java.util.Arrays;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
+import org.apache.carbondata.core.dictionary.server.DictionaryServer;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+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.util.CarbonProperties;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class to test dictionary client functionality.
+ */
+public class DictionaryClientTest {
+
+  private ColumnSchema empColumnSchema;
+  private CarbonDimension empDimension;
+  private ColumnSchema ageColumnSchema;
+  private CarbonDimension ageDimension;
+  private TableSchema tableSchema;
+  private TableInfo tableInfo;
+  private String storePath;
+  private DictionaryServer server;
+
+  @Before public void setUp() throws Exception {
+    // enable lru cache by setting cache size
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
+
+    // Create two column schemas and dimensions for the table
+    empColumnSchema = new ColumnSchema();
+    empColumnSchema.setColumnName("empNameCol");
+    empColumnSchema.setColumnUniqueId("empNameCol");
+    empColumnSchema.setDimensionColumn(true);
+    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
+
+    ageColumnSchema = new ColumnSchema();
+    ageColumnSchema.setColumnName("empNameCol");
+    ageColumnSchema.setColumnUniqueId("empNameCol");
+    ageColumnSchema.setDimensionColumn(true);
+    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0, 0);
+
+    // Create a Table
+    tableSchema = new TableSchema();
+    tableSchema.setTableName("TestTable");
+    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
+    CarbonMetadata metadata = CarbonMetadata.getInstance();
+
+    tableInfo = new TableInfo();
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTableUniqueName("TestTable");
+    tableInfo.setDatabaseName("test");
+    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
+    tableInfo.setStorePath(storePath);
+    CarbonTable carbonTable = new CarbonTable();
+    carbonTable.loadCarbonTable(tableInfo);
+
+    // Add the created table to metadata
+    metadata.addCarbonTable(carbonTable);
+
+    // Start the server for testing the client
+    server = new DictionaryServer();
+    server.startServer(5678);
+  }
+
+  @Test public void testClient() throws Exception {
+    DictionaryClient client = new DictionaryClient();
+    client.startClient("localhost", 5678);
+
+    Thread.sleep(1000);
+    // Create a dictionary key
+    DictionaryMessage empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("FirstKey");
+
+    // Test dictionary initialization call
+    empKey.setType(DictionaryMessageType.TABLE_INTIALIZATION);
+    client.getDictionary(empKey);
+
+    // Test dictionary generation
+    for (int count = 2; count <= 10000; count++) {
+      empKey.setType(DictionaryMessageType.DICT_GENERATION);
+      empKey.setData("FirstKey" + count);
+      DictionaryMessage val = client.getDictionary(empKey);
+      Assert.assertEquals(count, val.getDictionaryValue());
+    }
+    // Test size function
+    empKey.setType(DictionaryMessageType.SIZE);
+    DictionaryMessage val = client.getDictionary(empKey);
+    Assert.assertEquals(10000, val.getDictionaryValue());
+
+
+    client.shutDown();
+
+    // Shutdown the server
+    server.shutdown();
+  }
+
+  @After public void tearDown() {
+    // Cleanup created files
+    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
+    cleanUpDirectory(new File(storePath));
+  }
+
+  private void cleanUpDirectory(File path) {
+    File[] files = path.listFiles();
+    if (null == files) {
+      return;
+    }
+    for (File file : files) {
+      if (file.isDirectory()) cleanUpDirectory(file);
+      else file.delete();
+    }
+    path.delete();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/edaaccb2/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
new file mode 100644
index 0000000..5179305
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGeneratorTest.java
@@ -0,0 +1,201 @@
+/*
+ * 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.dictionary.generator;
+
+import java.io.File;
+import java.util.Arrays;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+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.util.CarbonProperties;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class to test incremental column dictionary generator functionality
+ */
+public class IncrementalColumnDictionaryGeneratorTest {
+
+  @Before public void setUp() throws Exception {
+    // enable lru cache by setting cache size
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
+  }
+
+  @Test public void generateKeyOnce() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+
+    // Create the generator and add the key to dictionary
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+    Integer key = generator.generateKey("First");
+    assertEquals(new Integer(11), key);
+  }
+
+  @Test public void generateKeyTwice() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+
+    // Create the generator and add the key to dictionary
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+    Integer key = generator.generateKey("First");
+
+    // Add one more key and check if it works fine.
+    key = generator.generateKey("Second");
+    assertEquals(new Integer(12), key);
+  }
+
+  @Test public void generateKeyAgain() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+
+    // Create the generator and add the key to dictionary
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+    Integer key = generator.generateKey("First");
+
+    // Add the same key again anc check if the value is correct
+    key = generator.generateKey("First");
+    assertEquals(new Integer(11), key);
+  }
+
+  @Test public void getKey() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+
+    // Create the generator and add the key to dictionary
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+    Integer generatedKey = generator.generateKey("First");
+
+    // Get the value of the key from dictionary and check if it matches with the created value
+    Integer obtainedKey = generator.getKey("First");
+    assertEquals(generatedKey, obtainedKey);
+  }
+
+  @Test public void getKeyInvalid() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+
+    // Try to get value for an invalid key
+    Integer obtainedKey = generator.getKey("Second");
+    assertNull(obtainedKey);
+  }
+
+  @Test public void getOrGenerateKey() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empName");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+
+    // Test first with generating a key and then trying geOrGenerate
+    Integer generatedKey = generator.generateKey("First");
+    Integer obtainedKey = generator.getOrGenerateKey("First");
+    assertEquals(generatedKey, obtainedKey);
+
+    // Test directly with getOrGenerate for another key
+    obtainedKey = generator.getOrGenerateKey("Second");
+    assertEquals(new Integer(12), obtainedKey);
+
+  }
+
+  @Test public void writeDictionaryData() throws Exception {
+    //Create required column schema
+    ColumnSchema columnSchema = new ColumnSchema();
+    columnSchema.setColumnName("empNameCol");
+    columnSchema.setColumnUniqueId("empNameCol");
+    CarbonDimension carbonDimension = new CarbonDimension(columnSchema, 0, 0, 0, 0, 0);
+
+    // Create the generator and add the keys to dictionary
+    IncrementalColumnDictionaryGenerator generator =
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+    Integer generatedKey = generator.generateKey("First");
+    Integer obtainedKey = generator.getOrGenerateKey("First");
+
+    // Create a table schema for saving the dictionary
+    TableSchema tableSchema = new TableSchema();
+    tableSchema.setTableName("TestTable");
+    tableSchema.setListOfColumns(Arrays.asList(columnSchema));
+    CarbonMetadata metadata = CarbonMetadata.getInstance();
+    TableInfo tableInfo = new TableInfo();
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTableUniqueName("TestTable");
+    tableInfo.setDatabaseName("test");
+
+    String storePath = System.getProperty("java.io.tmpdir") + "/tmp";
+    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
+    System.out.print(dictPath.mkdirs());
+
+    tableInfo.setStorePath(storePath);
+    CarbonTable carbonTable = new CarbonTable();
+    carbonTable.loadCarbonTable(tableInfo);
+
+    // Add the table to metadata
+    metadata.addCarbonTable(carbonTable);
+
+    /// Write the dictionary and verify whether its written successfully
+    generator.writeDictionaryData("TestTable");
+    File dictionaryFile = new File(dictPath, "empNameCol.dict");
+    System.out.println(dictionaryFile.getCanonicalPath());
+    assertTrue(dictionaryFile.exists());
+    dictionaryFile.delete();
+
+    // cleanup created files
+    metadata.removeTable(carbonTable.getTableUniqueName());
+    cleanUpDirectory(new File(storePath));
+
+  }
+
+  private void cleanUpDirectory(File path) {
+    File[] files = path.listFiles();
+    if (null == files) {
+      return;
+    }
+    for (File file : files) {
+      if (file.isDirectory()) cleanUpDirectory(file);
+      else file.delete();
+    }
+    path.delete();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/edaaccb2/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
new file mode 100644
index 0000000..1fe31a5
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGeneratorTest.java
@@ -0,0 +1,217 @@
+/*
+ * 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.dictionary.generator;
+
+import java.io.File;
+import java.util.Arrays;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+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.util.CarbonProperties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class to test server column dictionary generator functionality
+ */
+public class ServerDictionaryGeneratorTest {
+
+  private ColumnSchema empColumnSchema;
+  private CarbonDimension empDimension;
+  private ColumnSchema ageColumnSchema;
+  private CarbonDimension ageDimension;
+  private TableSchema tableSchema;
+  private TableInfo tableInfo;
+  private String storePath;
+
+  @Before public void setUp() throws Exception {
+    // enable lru cache by setting cache size
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
+
+    // Create two column schemas and dimensions for the table
+    empColumnSchema = new ColumnSchema();
+    empColumnSchema.setColumnName("empNameCol");
+    empColumnSchema.setColumnUniqueId("empNameCol");
+    empColumnSchema.setDimensionColumn(true);
+    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
+
+    ageColumnSchema = new ColumnSchema();
+    ageColumnSchema.setColumnName("empNameCol");
+    ageColumnSchema.setColumnUniqueId("empNameCol");
+    ageColumnSchema.setDimensionColumn(true);
+    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0, 0);
+
+    // Create a Table
+    tableSchema = new TableSchema();
+    tableSchema.setTableName("TestTable");
+    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
+    CarbonMetadata metadata = CarbonMetadata.getInstance();
+
+    tableInfo = new TableInfo();
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTableUniqueName("TestTable");
+    tableInfo.setDatabaseName("test");
+    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
+    tableInfo.setStorePath(storePath);
+    CarbonTable carbonTable = new CarbonTable();
+    carbonTable.loadCarbonTable(tableInfo);
+
+    // Add the created table to metadata
+    metadata.addCarbonTable(carbonTable);
+  }
+
+  @Test public void generateKeyOnce() throws Exception {
+    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
+
+    // Generate dictionary for one key
+    DictionaryMessage empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("FirstKey");
+    serverDictionaryGenerator.initializeGeneratorForTable(empKey);
+    Integer value = serverDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(2), value);
+
+  }
+
+  @Test public void generateKeyTwice() throws Exception {
+    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
+
+    // Generate dictionary for same key twice
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    Integer value = serverDictionaryGenerator.generateKey(firstKey);
+    assertEquals(new Integer(2), value);
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(empColumnSchema.getColumnName());
+    secondKey.setData("FirstKey");
+    value = serverDictionaryGenerator.generateKey(secondKey);
+    assertEquals(new Integer(2), value);
+  }
+
+  @Test public void generateKeyAgain() throws Exception {
+    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
+
+    // Generate dictionary for two different keys
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    Integer value = serverDictionaryGenerator.generateKey(firstKey);
+    assertEquals(new Integer(2), value);
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(empColumnSchema.getColumnName());
+    secondKey.setData("SecondKey");
+    value = serverDictionaryGenerator.generateKey(secondKey);
+    assertEquals(new Integer(3), value);
+  }
+
+  @Test public void size() throws Exception {
+    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
+    //Add keys for first Column
+    DictionaryMessage empKey = new DictionaryMessage();
+    //Add key 1
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("FirstKey");
+    serverDictionaryGenerator.initializeGeneratorForTable(empKey);
+    serverDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(2), serverDictionaryGenerator.size(empKey));
+
+    //Add key 2
+    empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("SecondKey");
+    serverDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(3), serverDictionaryGenerator.size(empKey));
+
+    //Add key 3
+    empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("ThirdKey");
+    serverDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(4), serverDictionaryGenerator.size(empKey));
+  }
+
+  @Test public void writeDictionaryData() throws Exception {
+    ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    Integer value = serverDictionaryGenerator.generateKey(firstKey);
+
+    //Update generator with a new dimension
+
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(ageColumnSchema.getColumnName());
+    secondKey.setData("SecondKey");
+    serverDictionaryGenerator.generateKey(secondKey);
+    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
+    System.out.print(dictPath.mkdirs());
+    serverDictionaryGenerator.writeDictionaryData();
+
+    File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
+    assertTrue(empDictionaryFile.exists());
+
+    File ageDictionaryFile = new File(dictPath, ageColumnSchema.getColumnName() + ".dict");
+    assertTrue(ageDictionaryFile.exists());
+  }
+
+  @After public void tearDown() {
+    cleanUpDirectory(new File(storePath));
+  }
+
+  private void cleanUpDirectory(File path) {
+    File[] files = path.listFiles();
+    if (null == files) {
+      return;
+    }
+    for (File file : files) {
+      if (file.isDirectory()) cleanUpDirectory(file);
+      else file.delete();
+    }
+    path.delete();
+    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/edaaccb2/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
new file mode 100644
index 0000000..c96a7d4
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGeneratorTest.java
@@ -0,0 +1,233 @@
+/*
+ * 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.dictionary.generator;
+
+import java.io.File;
+import java.util.Arrays;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+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.util.CarbonProperties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class to test table column dictionary generator functionality
+ */
+public class TableDictionaryGeneratorTest {
+
+  private ColumnSchema empColumnSchema;
+  private CarbonDimension empDimension;
+  private ColumnSchema ageColumnSchema;
+  private CarbonDimension ageDimension;
+  private TableSchema tableSchema;
+  private TableInfo tableInfo;
+  private String storePath;
+
+  @Before public void setUp() throws Exception {
+    // enable lru cache by setting cache size
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
+
+    // Create two column schemas and dimensions for the table
+    empColumnSchema = new ColumnSchema();
+    empColumnSchema.setColumnName("empNameCol");
+    empColumnSchema.setColumnUniqueId("empNameCol");
+    empColumnSchema.setDimensionColumn(true);
+    empColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
+
+    ageColumnSchema = new ColumnSchema();
+    ageColumnSchema.setColumnName("empNameCol");
+    ageColumnSchema.setColumnUniqueId("empNameCol");
+    ageColumnSchema.setDimensionColumn(true);
+    ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
+    ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0, 0);
+
+    // Create a Table
+    tableSchema = new TableSchema();
+    tableSchema.setTableName("TestTable");
+    tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
+    CarbonMetadata metadata = CarbonMetadata.getInstance();
+
+    tableInfo = new TableInfo();
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTableUniqueName("TestTable");
+    tableInfo.setDatabaseName("test");
+    storePath = System.getProperty("java.io.tmpdir") + "/tmp";
+    tableInfo.setStorePath(storePath);
+    CarbonTable carbonTable = new CarbonTable();
+    carbonTable.loadCarbonTable(tableInfo);
+
+    // Add the created table to metadata
+    metadata.addCarbonTable(carbonTable);
+  }
+
+  @Test public void generateKeyOnce() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+
+    // Generate dictionary for one key
+    DictionaryMessage empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("FirstKey");
+    Integer value = tableDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(2), value);
+
+  }
+
+  @Test public void generateKeyTwice() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+
+    // Generate dictionary for same key twice
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    Integer value = tableDictionaryGenerator.generateKey(firstKey);
+    assertEquals(new Integer(2), value);
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(empColumnSchema.getColumnName());
+    secondKey.setData("FirstKey");
+    value = tableDictionaryGenerator.generateKey(secondKey);
+    assertEquals(new Integer(2), value);
+  }
+
+  @Test public void generateKeyAgain() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+
+    // Generate dictionary for two different keys
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    Integer value = tableDictionaryGenerator.generateKey(firstKey);
+    assertEquals(new Integer(2), value);
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(empColumnSchema.getColumnName());
+    secondKey.setData("SecondKey");
+    value = tableDictionaryGenerator.generateKey(secondKey);
+    assertEquals(new Integer(3), value);
+  }
+
+  @Test public void updateGenerator() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    Integer value = tableDictionaryGenerator.generateKey(firstKey);
+    assertEquals(new Integer(2), value);
+
+    tableDictionaryGenerator.updateGenerator(ageDimension);
+    //Update generator with a new dimension
+
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(ageColumnSchema.getColumnName());
+    secondKey.setData("SecondKey");
+    value = tableDictionaryGenerator.generateKey(secondKey);
+    assertEquals(new Integer(2), value);
+  }
+
+  @Test public void size() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    //Add keys for first Column
+    DictionaryMessage empKey = new DictionaryMessage();
+    //Add key 1
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("FirstKey");
+    tableDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(2), tableDictionaryGenerator.size(empKey));
+
+    //Add key 2
+    empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("SecondKey");
+    tableDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(3), tableDictionaryGenerator.size(empKey));
+
+    //Add key 3
+    empKey = new DictionaryMessage();
+    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setData("ThirdKey");
+    tableDictionaryGenerator.generateKey(empKey);
+    assertEquals(new Integer(4), tableDictionaryGenerator.size(empKey));
+  }
+
+  @Test public void writeDictionaryData() throws Exception {
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    DictionaryMessage firstKey = new DictionaryMessage();
+    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setData("FirstKey");
+    Integer value = tableDictionaryGenerator.generateKey(firstKey);
+
+    tableDictionaryGenerator.updateGenerator(ageDimension);
+    //Update generator with a new dimension
+
+    DictionaryMessage secondKey = new DictionaryMessage();
+    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setColumnName(ageColumnSchema.getColumnName());
+    secondKey.setData("SecondKey");
+    tableDictionaryGenerator.generateKey(secondKey);
+    File dictPath = new File(storePath + "/test/TestTable/Metadata/");
+    System.out.print(dictPath.mkdirs());
+    tableDictionaryGenerator.writeDictionaryData(tableInfo.getTableUniqueName());
+
+    File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
+    assertTrue(empDictionaryFile.exists());
+
+    File ageDictionaryFile = new File(dictPath, ageColumnSchema.getColumnName() + ".dict");
+    assertTrue(ageDictionaryFile.exists());
+  }
+
+  @After public void tearDown() {
+    CarbonMetadata.getInstance().removeTable(tableInfo.getTableUniqueName());
+    cleanUpDirectory(new File(storePath));
+  }
+
+  private void cleanUpDirectory(File path) {
+    File[] files = path.listFiles();
+    if (null == files) {
+      return;
+    }
+    for (File file : files) {
+      if (file.isDirectory()) cleanUpDirectory(file);
+      else file.delete();
+    }
+    path.delete();
+  }
+}
\ No newline at end of file



[2/2] incubator-carbondata git commit: [CARBONDATA-684] Improve Tests and Coverage For Core Module This closes #578

Posted by ra...@apache.org.
[CARBONDATA-684] Improve Tests and Coverage For Core Module This closes #578


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/06f9cf6f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/06f9cf6f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/06f9cf6f

Branch: refs/heads/master
Commit: 06f9cf6f37e79538b9b4bb5f3bd28b943d614e82
Parents: b93173f edaaccb
Author: ravipesala <ra...@gmail.com>
Authored: Fri Mar 3 00:16:59 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Fri Mar 3 00:16:59 2017 +0530

----------------------------------------------------------------------
 .../dictionary/client/DictionaryClientTest.java | 152 ++++++++++++
 ...ncrementalColumnDictionaryGeneratorTest.java | 201 ++++++++++++++++
 .../ServerDictionaryGeneratorTest.java          | 217 +++++++++++++++++
 .../generator/TableDictionaryGeneratorTest.java | 233 +++++++++++++++++++
 4 files changed, 803 insertions(+)
----------------------------------------------------------------------