carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From gvram...@apache.org
Subject carbondata git commit: updated map for dictionary generator
Date Mon, 04 Sep 2017 05:30:47 GMT
Repository: carbondata
Updated Branches:
  refs/heads/master e2e6076e9 -> ee5f65f7d


updated map for dictionary generator


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

Branch: refs/heads/master
Commit: ee5f65f7d3f8aa73eaef13758b06cfffca9f31af
Parents: e2e6076
Author: kunal642 <kunalkapoor642@gmail.com>
Authored: Tue Jun 6 18:32:38 2017 +0530
Committer: Venkata Ramana G <ramana.gollamudi@huawei.com>
Committed: Mon Sep 4 10:59:23 2017 +0530

----------------------------------------------------------------------
 .../dictionary/generator/DictionaryWriter.java  |  2 +-
 .../IncrementalColumnDictionaryGenerator.java   | 11 ++--
 .../generator/ServerDictionaryGenerator.java    | 54 ++++++-----------
 .../generator/TableDictionaryGenerator.java     | 38 ++++++------
 .../generator/key/DictionaryMessage.java        | 42 ++++++-------
 .../generator/key/DictionaryMessageType.java    |  6 +-
 .../dictionary/server/DictionaryServer.java     | 21 +++----
 .../server/DictionaryServerHandler.java         | 29 ++++-----
 .../core/metadata/schema/table/CarbonTable.java |  5 +-
 .../dictionary/client/DictionaryClientTest.java | 32 +++++-----
 ...ncrementalColumnDictionaryGeneratorTest.java | 61 ++++++++-----------
 .../ServerDictionaryGeneratorTest.java          | 63 ++++++++++----------
 .../generator/TableDictionaryGeneratorTest.java | 63 +++++++++++---------
 .../spark/rdd/CarbonDataRDDFactory.scala        | 14 ++---
 .../execution/command/carbonTableSchema.scala   |  2 +-
 .../util/ExternalColumnDictionaryTestCase.scala |  3 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  7 +--
 .../execution/command/carbonTableSchema.scala   |  2 +-
 .../processing/datatypes/PrimitiveDataType.java | 13 ++--
 .../impl/DictionaryFieldConverterImpl.java      | 13 ++--
 .../converter/impl/FieldEncoderFactory.java     | 18 +++---
 .../converter/impl/RowConverterImpl.java        |  4 +-
 22 files changed, 225 insertions(+), 278 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
index 92d2efc..25704a0 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/DictionaryWriter.java
@@ -23,6 +23,6 @@ import java.io.IOException;
  */
 public interface DictionaryWriter {
 
-  void writeDictionaryData(String tableUniqueName) throws IOException;
+  void writeDictionaryData() throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
index 20802be..f361d06 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
@@ -33,7 +33,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -73,7 +72,11 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
 
   private CarbonDimension dimension;
 
-  public IncrementalColumnDictionaryGenerator(CarbonDimension dimension, int maxValue) {
+  private CarbonTable carbonTable;
+
+  public IncrementalColumnDictionaryGenerator(CarbonDimension dimension, int maxValue,
+      CarbonTable carbonTable) {
+    this.carbonTable = carbonTable;
     this.maxValue = maxValue;
     this.currentDictionarySize = maxValue;
     this.dimension = dimension;
@@ -111,10 +114,8 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
     }
   }
 
-  @Override public void writeDictionaryData(String tableUniqueName) throws IOException {
+  @Override public void writeDictionaryData() throws IOException {
     // initialize params
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-    CarbonTable carbonTable = metadata.getCarbonTable(tableUniqueName);
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
     CarbonTableIdentifier tableIdentifier = carbonTable.getCarbonTableIdentifier();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
index 221f9b7..0caa269 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/ServerDictionaryGenerator.java
@@ -16,16 +16,13 @@
  */
 package org.apache.carbondata.core.dictionary.generator;
 
-import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 
 /**
  * This is the dictionary generator for all tables. It generates dictionary
@@ -41,54 +38,41 @@ public class ServerDictionaryGenerator implements DictionaryGenerator<Integer, D
   @Override
   public Integer generateKey(DictionaryMessage value)
       throws DictionaryGenerationException {
-    TableDictionaryGenerator generator = tableMap.get(value.getTableUniqueName());
-    assert generator != null : "Table initialization for generator is not done";
+    initializeGeneratorForColumn(value);
+    TableDictionaryGenerator generator = tableMap.get(value.getTableUniqueId());
     return generator.generateKey(value);
   }
 
-  public void initializeGeneratorForTable(DictionaryMessage key) {
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-    CarbonTable carbonTable = metadata.getCarbonTable(key.getTableUniqueName());
-    CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(
-            key.getTableUniqueName(), key.getColumnName());
+  public void initializeGeneratorForTable(CarbonTable carbonTable) {
     // initialize TableDictionaryGenerator first
-    if (tableMap.get(key.getTableUniqueName()) == null) {
+    String tableId = carbonTable.getCarbonTableIdentifier().getTableId();
+    if (tableMap.get(tableId) == null) {
       synchronized (tableMap) {
-        if (tableMap.get(key.getTableUniqueName()) == null) {
-          tableMap.put(key.getTableUniqueName(), new TableDictionaryGenerator(dimension));
-        } else {
-          tableMap.get(key.getTableUniqueName()).updateGenerator(dimension);
+        if (tableMap.get(tableId) == null) {
+          tableMap.put(tableId,
+              new TableDictionaryGenerator(carbonTable));
         }
       }
-    } else {
-      tableMap.get(key.getTableUniqueName()).updateGenerator(dimension);
     }
   }
 
-  public Integer size(DictionaryMessage key) {
-    TableDictionaryGenerator generator = tableMap.get(key.getTableUniqueName());
-    assert generator != null : "Table intialization for generator is not done";
-    return generator.size(key);
+  public void initializeGeneratorForColumn(DictionaryMessage key) {
+    tableMap.get(key.getTableUniqueId()).updateGenerator(key);
   }
 
-  public void writeDictionaryData() throws Exception {
-    final Iterator<Map.Entry<String, TableDictionaryGenerator>> iterator =
-        tableMap.entrySet().iterator();
-    String tableUniqueName;
-    TableDictionaryGenerator generator;
-    while (iterator.hasNext()) {
-      Map.Entry<String, TableDictionaryGenerator> entry = iterator.next();
-      tableUniqueName = entry.getKey();
-      generator = entry.getValue();
-      generator.writeDictionaryData(tableUniqueName);
-    }
+  public Integer size(DictionaryMessage key) {
+    initializeGeneratorForColumn(key);
+    TableDictionaryGenerator generator = tableMap.get(key.getTableUniqueId());
+    return generator.size(key);
   }
 
-  public void writeTableDictionaryData(String tableUniqueName) throws Exception {
-    TableDictionaryGenerator generator = tableMap.get(tableUniqueName);
+  public void writeTableDictionaryData(String tableUniqueId) throws Exception {
+    TableDictionaryGenerator generator = tableMap.get(tableUniqueId);
     if (generator != null) {
-      generator.writeDictionaryData(tableUniqueName);
+      generator.writeDictionaryData();
     }
+    // Remove dictionary generator after writing
+    tableMap.remove(tableUniqueId);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
index ee3a2fe..f08ba1f 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.devapi.BiDictionary;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.devapi.DictionaryGenerator;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -44,23 +43,20 @@ public class TableDictionaryGenerator
   private static final LogService LOGGER =
           LogServiceFactory.getLogService(TableDictionaryGenerator.class.getName());
 
+  private CarbonTable carbonTable;
   /**
    * the map of columnName to dictionaryGenerator
    */
   private Map<String, DictionaryGenerator<Integer, String>> columnMap = new ConcurrentHashMap<>();
 
-  public TableDictionaryGenerator(CarbonDimension dimension) {
-    columnMap.put(dimension.getColumnId(),
-            new IncrementalColumnDictionaryGenerator(dimension, 1));
+  public TableDictionaryGenerator(CarbonTable carbonTable) {
+    this.carbonTable = carbonTable;
   }
 
   @Override
   public Integer generateKey(DictionaryMessage value)
       throws DictionaryGenerationException {
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-    CarbonTable carbonTable = metadata.getCarbonTable(value.getTableUniqueName());
-    CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(
-            value.getTableUniqueName(), value.getColumnName());
+    CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(value.getColumnName());
 
     DictionaryGenerator<Integer, String> generator =
             columnMap.get(dimension.getColumnId());
@@ -68,17 +64,14 @@ public class TableDictionaryGenerator
   }
 
   public Integer size(DictionaryMessage key) {
-    CarbonMetadata metadata = CarbonMetadata.getInstance();
-    CarbonTable carbonTable = metadata.getCarbonTable(key.getTableUniqueName());
-    CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(
-            key.getTableUniqueName(), key.getColumnName());
+    CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(key.getColumnName());
 
     DictionaryGenerator<Integer, String> generator =
             columnMap.get(dimension.getColumnId());
     return ((BiDictionary) generator).size();
   }
 
-  @Override public void writeDictionaryData(String tableUniqueName) {
+  @Override public void writeDictionaryData() {
     int numOfCores = 1;
     try {
       numOfCores = Integer.parseInt(CarbonProperties.getInstance()
@@ -90,7 +83,7 @@ public class TableDictionaryGenerator
     long start = System.currentTimeMillis();
     ExecutorService executorService = Executors.newFixedThreadPool(numOfCores);
     for (final DictionaryGenerator generator : columnMap.values()) {
-      executorService.execute(new WriteDictionaryDataRunnable(generator, tableUniqueName));
+      executorService.execute(new WriteDictionaryDataRunnable(generator));
     }
 
     try {
@@ -103,13 +96,14 @@ public class TableDictionaryGenerator
             (System.currentTimeMillis() - start));
   }
 
-  public void updateGenerator(CarbonDimension dimension) {
-    // reuse dictionary generator
+  public void updateGenerator(DictionaryMessage key) {
+    CarbonDimension dimension = carbonTable
+        .getPrimitiveDimensionByName(key.getColumnName());
     if (null == columnMap.get(dimension.getColumnId())) {
       synchronized (columnMap) {
         if (null == columnMap.get(dimension.getColumnId())) {
           columnMap.put(dimension.getColumnId(),
-              new IncrementalColumnDictionaryGenerator(dimension, 1));
+              new IncrementalColumnDictionaryGenerator(dimension, 1, carbonTable));
         }
       }
     }
@@ -117,19 +111,21 @@ public class TableDictionaryGenerator
 
   private static class WriteDictionaryDataRunnable implements Runnable {
     private final DictionaryGenerator generator;
-    private final String tableUniqueName;
 
-    public WriteDictionaryDataRunnable(DictionaryGenerator generator, String tableUniqueName) {
+    public WriteDictionaryDataRunnable(DictionaryGenerator generator) {
       this.generator = generator;
-      this.tableUniqueName = tableUniqueName;
     }
 
     @Override public void run() {
       try {
-        ((DictionaryWriter)generator).writeDictionaryData(tableUniqueName);
+        ((DictionaryWriter)generator).writeDictionaryData();
       } catch (IOException e) {
         throw new RuntimeException(e);
       }
     }
   }
+  public String getTableUniqueName() {
+    return carbonTable.getTableUniqueName();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
index 0bb82f9..749c3f4 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessage.java
@@ -28,9 +28,9 @@ import io.netty.buffer.ByteBuf;
 public class DictionaryMessage {
 
   /**
-   * tableUniqueName
+   * tableUniqueId
    */
-  private String tableUniqueName;
+  private String tableUniqueId;
 
   /**
    * columnName
@@ -53,10 +53,9 @@ public class DictionaryMessage {
   private DictionaryMessageType type;
 
   public void readData(ByteBuf byteBuf) {
-    byte[] tableBytes = new byte[byteBuf.readInt()];
-    byteBuf.readBytes(tableBytes);
-    tableUniqueName =
-        new String(tableBytes, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+    byte[] tableIdBytes = new byte[byteBuf.readInt()];
+    byteBuf.readBytes(tableIdBytes);
+    tableUniqueId = new String(tableIdBytes);
 
     byte[] colBytes = new byte[byteBuf.readInt()];
     byteBuf.readBytes(colBytes);
@@ -80,10 +79,9 @@ public class DictionaryMessage {
     // Just reserve the bytes to add length of header at last.
     byteBuf.writeShort(Short.MAX_VALUE);
 
-    byte[] tableBytes =
-        tableUniqueName.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    byteBuf.writeInt(tableBytes.length);
-    byteBuf.writeBytes(tableBytes);
+    byte[] tableIdBytes = tableUniqueId.getBytes();
+    byteBuf.writeInt(tableIdBytes.length);
+    byteBuf.writeBytes(tableIdBytes);
 
     byte[] colBytes = columnName.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
     byteBuf.writeInt(colBytes.length);
@@ -109,23 +107,15 @@ public class DictionaryMessage {
 
   private DictionaryMessageType getKeyType(byte type) {
     switch (type) {
-      case 1:
-        return DictionaryMessageType.DICT_GENERATION;
       case 2:
-        return DictionaryMessageType.TABLE_INTIALIZATION;
-      case 3:
         return DictionaryMessageType.SIZE;
-      case 4:
-        return DictionaryMessageType.WRITE_DICTIONARY;
+      case 3:
+        return DictionaryMessageType.WRITE_TABLE_DICTIONARY;
       default:
         return DictionaryMessageType.DICT_GENERATION;
     }
   }
 
-  public String getTableUniqueName() {
-    return tableUniqueName;
-  }
-
   public String getColumnName() {
     return columnName;
   }
@@ -146,10 +136,6 @@ public class DictionaryMessage {
     this.type = type;
   }
 
-  public void setTableUniqueName(String tableUniqueName) {
-    this.tableUniqueName = tableUniqueName;
-  }
-
   public void setColumnName(String columnName) {
     this.columnName = columnName;
   }
@@ -162,6 +148,14 @@ public class DictionaryMessage {
     this.dictionaryValue = dictionaryValue;
   }
 
+  public String getTableUniqueId() {
+    return tableUniqueId;
+  }
+
+  public void setTableUniqueId(String tableUniqueId) {
+    this.tableUniqueId = tableUniqueId;
+  }
+
   @Override public String toString() {
     return "DictionaryKey{ columnName='" + columnName + '\'' + ", data='" + data + '\''
         + ", dictionaryValue=" + dictionaryValue + ", type=" + type + '}';

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
index b3d1f9a..c87df06 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/key/DictionaryMessageType.java
@@ -22,10 +22,8 @@ package org.apache.carbondata.core.dictionary.generator.key;
 public enum DictionaryMessageType {
 
   DICT_GENERATION((byte) 1),
-  TABLE_INTIALIZATION((byte) 2),
-  SIZE((byte) 3),
-  WRITE_DICTIONARY((byte) 4),
-  WRITE_TABLE_DICTIONARY((byte) 5);
+  SIZE((byte) 2),
+  WRITE_TABLE_DICTIONARY((byte) 3);
 
   final byte type;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
index 617038e..d8a7c9e 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServer.java
@@ -21,6 +21,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 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.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 import io.netty.bootstrap.ServerBootstrap;
@@ -53,7 +54,7 @@ public class DictionaryServer {
     startServer(port);
   }
 
-  public static DictionaryServer getInstance(int port) {
+  public static DictionaryServer getInstance(int port, CarbonTable carbonTable) throws Exception {
     if (INSTANCE == null) {
       synchronized (lock) {
         if (INSTANCE == null) {
@@ -61,6 +62,7 @@ public class DictionaryServer {
         }
       }
     }
+    INSTANCE.initializeDictionaryGenerator(carbonTable);
     return INSTANCE;
   }
 
@@ -140,26 +142,17 @@ public class DictionaryServer {
     boss.shutdownGracefully();
   }
 
-
-
-  /**
-   * Write dictionary to the store.
-   * @throws Exception
-   */
-  public void writeDictionary() throws Exception {
-    DictionaryMessage key = new DictionaryMessage();
-    key.setType(DictionaryMessageType.WRITE_DICTIONARY);
-    dictionaryServerHandler.processMessage(key);
+  public void initializeDictionaryGenerator(CarbonTable carbonTable) throws Exception {
+    dictionaryServerHandler.initializeTable(carbonTable);
   }
 
   /**
    *  Write Dictionary for one table.
    * @throws Exception
    */
-
-  public void writeTableDictionary(String uniqueTableName) throws Exception {
+  public void writeTableDictionary(String tableId) throws Exception {
     DictionaryMessage key = new DictionaryMessage();
-    key.setTableUniqueName(uniqueTableName);
+    key.setTableUniqueId(tableId);
     key.setType(DictionaryMessageType.WRITE_TABLE_DICTIONARY);
     dictionaryServerHandler.processMessage(key);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
index 946a43d..cf8581b 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/DictionaryServerHandler.java
@@ -20,6 +20,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.dictionary.generator.ServerDictionaryGenerator;
 import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandler;
@@ -29,11 +30,10 @@ import io.netty.channel.ChannelInboundHandlerAdapter;
 /**
  * Handler for Dictionary server.
  */
-@ChannelHandler.Sharable
-public class DictionaryServerHandler extends ChannelInboundHandlerAdapter {
+@ChannelHandler.Sharable public class DictionaryServerHandler extends ChannelInboundHandlerAdapter {
 
   private static final LogService LOGGER =
-          LogServiceFactory.getLogService(DictionaryServerHandler.class.getName());
+      LogServiceFactory.getLogService(DictionaryServerHandler.class.getName());
 
   /**
    * dictionary generator
@@ -51,8 +51,7 @@ public class DictionaryServerHandler extends ChannelInboundHandlerAdapter {
     super.channelActive(ctx);
   }
 
-  @Override
-  public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+  @Override public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
     try {
       ByteBuf data = (ByteBuf) msg;
       DictionaryMessage key = new DictionaryMessage();
@@ -76,8 +75,7 @@ public class DictionaryServerHandler extends ChannelInboundHandlerAdapter {
    * @param ctx
    * @param cause
    */
-  @Override
-  public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
+  @Override public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
     LOGGER.error(cause, "exceptionCaught");
     ctx.close();
   }
@@ -91,22 +89,21 @@ public class DictionaryServerHandler extends ChannelInboundHandlerAdapter {
    */
   public int processMessage(DictionaryMessage key) throws Exception {
     switch (key.getType()) {
-      case DICT_GENERATION :
+      case DICT_GENERATION:
         return generatorForServer.generateKey(key);
-      case TABLE_INTIALIZATION :
-        generatorForServer.initializeGeneratorForTable(key);
-        return 0;
-      case SIZE :
+      case SIZE:
         return generatorForServer.size(key);
-      case WRITE_DICTIONARY :
-        generatorForServer.writeDictionaryData();
-        return 0;
       case WRITE_TABLE_DICTIONARY:
-        generatorForServer.writeTableDictionaryData(key.getTableUniqueName());
+        generatorForServer
+            .writeTableDictionaryData(key.getTableUniqueId());
         return 0;
       default:
         return -1;
     }
   }
 
+  void initializeTable(CarbonTable carbonTable) {
+    generatorForServer.initializeGeneratorForTable(carbonTable);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/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 07d856c..20f3224 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
@@ -572,11 +572,10 @@ public class CarbonTable implements Serializable {
   /**
    * to get the normal dimension or the primitive dimension of the complex type
    *
-   * @param tableName
    * @return primitive dimension of a table
    */
-  public CarbonDimension getPrimitiveDimensionByName(String tableName, String columnName) {
-    List<CarbonDimension> dimList = tablePrimitiveDimensionsMap.get(tableName);
+  public CarbonDimension getPrimitiveDimensionByName(String columnName) {
+    List<CarbonDimension> dimList = tablePrimitiveDimensionsMap.get(tableUniqueName);
     for (CarbonDimension dim : dimList) {
       if (!dim.isInvisible() && dim.getColName().equalsIgnoreCase(columnName)) {
         return dim;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/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
index dc3b232..34906c3 100644
--- 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
@@ -40,6 +40,7 @@ import mockit.MockUp;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
@@ -47,16 +48,16 @@ import org.junit.Test;
  */
 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;
+  private static ColumnSchema empColumnSchema;
+  private static CarbonDimension empDimension;
+  private static ColumnSchema ageColumnSchema;
+  private static CarbonDimension ageDimension;
+  private static TableSchema tableSchema;
+  private static TableInfo tableInfo;
+  private static String storePath;
+  private static DictionaryServer server;
 
-  @Before public void setUp() throws Exception {
+  @BeforeClass public static void setUp() throws Exception {
     // enable lru cache by setting cache size
     CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
@@ -70,8 +71,8 @@ public class DictionaryClientTest {
     empDimension = new CarbonDimension(empColumnSchema, 0, 0, 0, 0, 0);
 
     ageColumnSchema = new ColumnSchema();
-    ageColumnSchema.setColumnName("empNameCol");
-    ageColumnSchema.setColumnUniqueId("empNameCol");
+    ageColumnSchema.setColumnName("ageNameCol");
+    ageColumnSchema.setColumnUniqueId("ageNameCol");
     ageColumnSchema.setDimensionColumn(true);
     ageColumnSchema.setEncodingList(Arrays.asList(Encoding.DICTIONARY));
     ageDimension = new CarbonDimension(ageColumnSchema, 0, 0, 0, 0, 0);
@@ -79,6 +80,7 @@ public class DictionaryClientTest {
     // Create a Table
     tableSchema = new TableSchema();
     tableSchema.setTableName("TestTable");
+    tableSchema.setTableId("1");
     tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
     CarbonMetadata metadata = CarbonMetadata.getInstance();
 
@@ -94,7 +96,7 @@ public class DictionaryClientTest {
     metadata.addCarbonTable(carbonTable);
 
     // Start the server for testing the client
-    server = DictionaryServer.getInstance(5678);
+    server = DictionaryServer.getInstance(5678, carbonTable);
   }
 
   @Test public void testClient() throws Exception {
@@ -104,17 +106,15 @@ public class DictionaryClientTest {
     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);
     int count = 2;
     // Test dictionary generation
     for (; count <= 10000; count++) {
       empKey.setType(DictionaryMessageType.DICT_GENERATION);
+      empKey.setTableUniqueId("1");
       empKey.setData("FirstKey" + count);
       DictionaryMessage val = client.getDictionary(empKey);
       Assert.assertEquals(count, val.getDictionaryValue());
@@ -185,7 +185,7 @@ public class DictionaryClientTest {
     cleanUpDirectory(new File(storePath));
   }
 
-  private void cleanUpDirectory(File path) {
+  private static void cleanUpDirectory(File path) {
     File[] files = path.listFiles();
     if (null == files) {
       return;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/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
index 0bac01a..6f09897 100644
--- 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
@@ -41,34 +41,39 @@ import static org.junit.Assert.assertTrue;
  */
 public class IncrementalColumnDictionaryGeneratorTest {
 
+  private CarbonTable carbonTable;
+  private CarbonDimension carbonDimension;
+
   @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);
+    TableSchema tableSchema = new TableSchema();
+    tableSchema.setTableName("TestTable");
+    tableSchema.setListOfColumns(Arrays.asList(columnSchema));
+    TableInfo tableInfo = new TableInfo();
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTableUniqueName("TestTable");
+    tableInfo.setDatabaseName("test");
+    String storePath = System.getProperty("java.io.tmpdir") + "/tmp";
+    tableInfo.setStorePath(storePath);
+    carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
+    carbonDimension = new CarbonDimension(columnSchema,0,0,0,0,0);
+  }
 
+  @Test public void generateKeyOnce() throws Exception {
     // Create the generator and add the key to dictionary
     IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
     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);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
     Integer key = generator.generateKey("First");
 
     // Add one more key and check if it works fine.
@@ -77,14 +82,9 @@ public class IncrementalColumnDictionaryGeneratorTest {
   }
 
   @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);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
     Integer key = generator.generateKey("First");
 
     // Add the same key again anc check if the value is correct
@@ -93,14 +93,9 @@ public class IncrementalColumnDictionaryGeneratorTest {
   }
 
   @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);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
     Integer generatedKey = generator.generateKey("First");
 
     // Get the value of the key from dictionary and check if it matches with the created value
@@ -109,12 +104,8 @@ public class IncrementalColumnDictionaryGeneratorTest {
   }
 
   @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);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
 
     // Try to get value for an invalid key
     Integer obtainedKey = generator.getKey("Second");
@@ -122,12 +113,8 @@ public class IncrementalColumnDictionaryGeneratorTest {
   }
 
   @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);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
 
     // Test first with generating a key and then trying geOrGenerate
     Integer generatedKey = generator.generateKey("First");
@@ -149,7 +136,7 @@ public class IncrementalColumnDictionaryGeneratorTest {
 
     // Create the generator and add the keys to dictionary
     IncrementalColumnDictionaryGenerator generator =
-        new IncrementalColumnDictionaryGenerator(carbonDimension, 10);
+        new IncrementalColumnDictionaryGenerator(carbonDimension, 10, carbonTable);
 
     // Create a table schema for saving the dictionary
     TableSchema tableSchema = new TableSchema();
@@ -172,7 +159,7 @@ public class IncrementalColumnDictionaryGeneratorTest {
     metadata.addCarbonTable(carbonTable);
 
     /// Write the dictionary and verify whether its written successfully
-    generator.writeDictionaryData("TestTable");
+    generator.writeDictionaryData();
     File dictionaryFile = new File(dictPath, "empNameCol.dict");
     System.out.println(dictionaryFile.getCanonicalPath());
     assertTrue(dictionaryFile.exists());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/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
index d8df99a..076968c 100644
--- 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
@@ -33,6 +33,7 @@ import org.apache.carbondata.core.util.CarbonProperties;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -43,15 +44,16 @@ import static org.junit.Assert.assertTrue;
  */
 public class ServerDictionaryGeneratorTest {
 
-  private ColumnSchema empColumnSchema;
-  private CarbonDimension empDimension;
-  private ColumnSchema ageColumnSchema;
-  private CarbonDimension ageDimension;
-  private TableSchema tableSchema;
-  private TableInfo tableInfo;
-  private String storePath;
+  private static ColumnSchema empColumnSchema;
+  private static CarbonDimension empDimension;
+  private static ColumnSchema ageColumnSchema;
+  private static CarbonDimension ageDimension;
+  private static TableSchema tableSchema;
+  private static TableInfo tableInfo;
+  private static String storePath;
+  private static CarbonTable carbonTable;
 
-  @Before public void setUp() throws Exception {
+  @BeforeClass public static void setUp() throws Exception {
     // enable lru cache by setting cache size
     CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE, "10");
@@ -74,6 +76,7 @@ public class ServerDictionaryGeneratorTest {
     // Create a Table
     tableSchema = new TableSchema();
     tableSchema.setTableName("TestTable");
+    tableSchema.setTableId("1");
     tableSchema.setListOfColumns(Arrays.asList(empColumnSchema, ageColumnSchema));
     CarbonMetadata metadata = CarbonMetadata.getInstance();
 
@@ -83,24 +86,21 @@ public class ServerDictionaryGeneratorTest {
     tableInfo.setDatabaseName("test");
     storePath = System.getProperty("java.io.tmpdir") + "/tmp";
     tableInfo.setStorePath(storePath);
-    CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-
+    carbonTable = CarbonTable.buildFromTableInfo(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.setTableUniqueId("1");
     empKey.setColumnName(empColumnSchema.getColumnName());
     empKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(empKey);
+    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
     Integer value = serverDictionaryGenerator.generateKey(empKey);
     assertEquals(new Integer(2), value);
-
   }
 
   @Test public void generateKeyTwice() throws Exception {
@@ -108,33 +108,32 @@ public class ServerDictionaryGeneratorTest {
 
     // Generate dictionary for same key twice
     DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
     firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setTableUniqueId("1");
     firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
     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");
+    secondKey.setTableUniqueId("1");
     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.setTableUniqueId("1");
     firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
     Integer value = serverDictionaryGenerator.generateKey(firstKey);
     assertEquals(new Integer(2), value);
     DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    secondKey.setTableUniqueId("1");
     secondKey.setColumnName(empColumnSchema.getColumnName());
     secondKey.setData("SecondKey");
     value = serverDictionaryGenerator.generateKey(secondKey);
@@ -146,16 +145,16 @@ public class ServerDictionaryGeneratorTest {
     //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);
+    empKey.setTableUniqueId("1");
+    empKey.setColumnName(ageColumnSchema.getColumnName());
+    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
     serverDictionaryGenerator.generateKey(empKey);
     assertEquals(new Integer(2), serverDictionaryGenerator.size(empKey));
 
     //Add key 2
     empKey = new DictionaryMessage();
-    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
+    empKey.setTableUniqueId("1");
     empKey.setColumnName(empColumnSchema.getColumnName());
     empKey.setData("SecondKey");
     serverDictionaryGenerator.generateKey(empKey);
@@ -163,8 +162,8 @@ public class ServerDictionaryGeneratorTest {
 
     //Add key 3
     empKey = new DictionaryMessage();
-    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
     empKey.setColumnName(empColumnSchema.getColumnName());
+    empKey.setTableUniqueId("1");
     empKey.setData("ThirdKey");
     serverDictionaryGenerator.generateKey(empKey);
     assertEquals(new Integer(4), serverDictionaryGenerator.size(empKey));
@@ -173,21 +172,21 @@ public class ServerDictionaryGeneratorTest {
   @Test public void writeDictionaryData() throws Exception {
     ServerDictionaryGenerator serverDictionaryGenerator = new ServerDictionaryGenerator();
     DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
     firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setTableUniqueId("1");
     firstKey.setData("FirstKey");
-    serverDictionaryGenerator.initializeGeneratorForTable(firstKey);
+    serverDictionaryGenerator.initializeGeneratorForTable(carbonTable);
 
     //Update generator with a new dimension
 
     DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
     secondKey.setColumnName(ageColumnSchema.getColumnName());
+    secondKey.setTableUniqueId("1");
     secondKey.setData("SecondKey");
     serverDictionaryGenerator.generateKey(secondKey);
     File dictPath = new File(storePath + "/test/TestTable/Metadata/");
-    System.out.print(dictPath.mkdirs());
-    serverDictionaryGenerator.writeDictionaryData();
+    dictPath.mkdirs();
+    serverDictionaryGenerator.writeTableDictionaryData("1");
 
     File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
     assertTrue(empDictionaryFile.exists());
@@ -200,7 +199,7 @@ public class ServerDictionaryGeneratorTest {
     cleanUpDirectory(new File(storePath));
   }
 
-  private void cleanUpDirectory(File path) {
+  private static void cleanUpDirectory(File path) {
     File[] files = path.listFiles();
     if (null == files) {
       return;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/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
index 8a68b72..184ed3c 100644
--- 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
@@ -22,6 +22,7 @@ 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.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -50,6 +51,7 @@ public class TableDictionaryGeneratorTest {
   private TableSchema tableSchema;
   private TableInfo tableInfo;
   private String storePath;
+  private CarbonTable carbonTable;
 
   @Before public void setUp() throws Exception {
     // enable lru cache by setting cache size
@@ -83,128 +85,131 @@ public class TableDictionaryGeneratorTest {
     tableInfo.setDatabaseName("test");
     storePath = System.getProperty("java.io.tmpdir") + "/tmp";
     tableInfo.setStorePath(storePath);
-    CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
-
+    carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
     // Add the created table to metadata
     metadata.addCarbonTable(carbonTable);
   }
 
   @Test public void generateKeyOnce() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
 
     // Generate dictionary for one key
     DictionaryMessage empKey = new DictionaryMessage();
-    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
     empKey.setColumnName(empColumnSchema.getColumnName());
     empKey.setData("FirstKey");
+    tableDictionaryGenerator.updateGenerator(empKey);
     Integer value = tableDictionaryGenerator.generateKey(empKey);
     assertEquals(new Integer(2), value);
 
   }
 
   @Test public void generateKeyTwice() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
 
     // Generate dictionary for same key twice
     DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
     firstKey.setColumnName(empColumnSchema.getColumnName());
+    firstKey.setTableUniqueId("1");
     firstKey.setData("FirstKey");
+    tableDictionaryGenerator.updateGenerator(firstKey);
     Integer value = tableDictionaryGenerator.generateKey(firstKey);
     assertEquals(new Integer(2), value);
     DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
     secondKey.setColumnName(empColumnSchema.getColumnName());
+    secondKey.setTableUniqueId("1");
     secondKey.setData("FirstKey");
     value = tableDictionaryGenerator.generateKey(secondKey);
     assertEquals(new Integer(2), value);
   }
 
   @Test public void generateKeyAgain() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
 
     // Generate dictionary for two different keys
     DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
     firstKey.setColumnName(empColumnSchema.getColumnName());
     firstKey.setData("FirstKey");
+    firstKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(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");
+    secondKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(secondKey);
     value = tableDictionaryGenerator.generateKey(secondKey);
     assertEquals(new Integer(3), value);
   }
 
   @Test public void updateGenerator() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
     DictionaryMessage firstKey = new DictionaryMessage();
-    firstKey.setTableUniqueName(tableInfo.getTableUniqueName());
     firstKey.setColumnName(empColumnSchema.getColumnName());
     firstKey.setData("FirstKey");
+    firstKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(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");
+    secondKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(secondKey);
+    //Update generator with a new dimension
     value = tableDictionaryGenerator.generateKey(secondKey);
     assertEquals(new Integer(2), value);
   }
 
   @Test public void size() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
     //Add keys for first Column
     DictionaryMessage empKey = new DictionaryMessage();
     //Add key 1
-    empKey.setTableUniqueName(tableInfo.getTableUniqueName());
     empKey.setColumnName(empColumnSchema.getColumnName());
     empKey.setData("FirstKey");
+    empKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(empKey);
     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");
+    empKey.setTableUniqueId("1");
     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");
+    empKey.setTableUniqueId("1");
     tableDictionaryGenerator.generateKey(empKey);
     assertEquals(new Integer(4), tableDictionaryGenerator.size(empKey));
   }
 
   @Test public void writeDictionaryData() throws Exception {
-    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(empDimension);
+    TableDictionaryGenerator tableDictionaryGenerator = new TableDictionaryGenerator(carbonTable);
     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
+    firstKey.setTableUniqueId("1");
+    tableDictionaryGenerator.updateGenerator(firstKey);
+    tableDictionaryGenerator.generateKey(firstKey);
 
     DictionaryMessage secondKey = new DictionaryMessage();
-    secondKey.setTableUniqueName(tableInfo.getTableUniqueName());
     secondKey.setColumnName(ageColumnSchema.getColumnName());
     secondKey.setData("SecondKey");
+    secondKey.setTableUniqueId("1");
+    //Update generator with a new dimension
+    tableDictionaryGenerator.updateGenerator(secondKey);
     tableDictionaryGenerator.generateKey(secondKey);
     File dictPath = new File(storePath + "/test/TestTable/Metadata/");
-    System.out.print(dictPath.mkdirs());
-    tableDictionaryGenerator.writeDictionaryData(tableInfo.getTableUniqueName());
+    dictPath.mkdirs();
+    tableDictionaryGenerator.writeDictionaryData();
 
     File empDictionaryFile = new File(dictPath, empColumnSchema.getColumnName() + ".dict");
     assertTrue(empDictionaryFile.exists());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 669f942..944c320 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -36,7 +36,7 @@ import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer, N
 import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
 import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, ExecutionErrors, UpdateTableModel}
 import org.apache.spark.sql.hive.DistributionUtil
-import org.apache.spark.util.SparkUtil
+import org.apache.spark.util.{FileUtils, SparkUtil}
 
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -905,7 +905,7 @@ object CarbonDataRDDFactory {
         }
         val metadataDetails = status(0)._2._1
         if (!isAgg) {
-            writeDictionary(carbonLoadModel, result, false)
+            writeDictionary(carbonLoadModel, result)
             CarbonLoaderUtil
               .populateNewLoadMetaEntry(metadataDetails,
                 loadStatus,
@@ -1059,19 +1059,15 @@ object CarbonDataRDDFactory {
   }
 
     private def writeDictionary(carbonLoadModel: CarbonLoadModel,
-        result: Option[DictionaryServer], writeAll: Boolean) = {
+        result: Option[DictionaryServer]) = {
     // write dictionary file and shutdown dictionary server
     val uniqueTableName: String = s"${ carbonLoadModel.getDatabaseName }_${
       carbonLoadModel.getTableName }"
     result match {
       case Some(server) =>
         try {
-          if (writeAll) {
-            server.writeDictionary()
-          }
-          else {
-            server.writeTableDictionary(uniqueTableName)
-          }
+          server.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+            .getCarbonTableIdentifier.getTableId)
         } catch {
           case ex: Exception =>
             LOGGER.error(s"Error while writing dictionary file for $uniqueTableName")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 00dfaec..c07188a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -607,7 +607,7 @@ case class LoadTable(
           }
           val server: Option[DictionaryServer] = if (createDictionary) {
             val dictionaryServer = DictionaryServer
-              .getInstance(dictionaryServerPort.toInt)
+              .getInstance(dictionaryServerPort.toInt, carbonTable)
             carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
             sqlContext.sparkContext.addSparkListener(new SparkListener() {
               override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 027086e..63f8c1f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -58,8 +58,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
       s"locationInfo.ActiveCountry:${resourcesPath}/locationInfoActiveCountry.csv"
     extColDictFilePath2 = s"deviceInformationId:${resourcesPath}/deviceInformationId2.csv"
     extColDictFilePath3 = s"channelsId:${resourcesPath}/channelsId.csv"
-    header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC," +
-      "locationinfo,proddate,gamePointId,contractNumber"
+    header = "deviceInformationId,channelsId,ROMSize,purchasedate,mobile,MAC,locationinfo,proddate,gamePointId,contractNumber"
     header2 = "deviceInformationId,channelsId,contractNumber"
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 34872b2..185066e 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -1172,11 +1172,8 @@ object CarbonDataRDDFactory {
     result match {
       case Some(server) =>
         try {
-          if (writeAll) {
-            server.writeDictionary()
-          } else {
-            server.writeTableDictionary(uniqueTableName)
-          }
+          server.writeTableDictionary(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+            .getCarbonTableIdentifier.getTableId)
         } catch {
           case _: Exception =>
             LOGGER.error(s"Error while writing dictionary file for $uniqueTableName")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 9cc12cd..5ae4702 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -837,7 +837,7 @@ case class LoadTable(
           }
           val server: Option[DictionaryServer] = if (createDictionary) {
             val dictionaryServer = DictionaryServer
-                .getInstance(dictionaryServerPort.toInt)
+              .getInstance(dictionaryServerPort.toInt, carbonTable)
             carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
             sparkSession.sparkContext.addSparkListener(new SparkListener() {
               override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index 8373b58..c6fc1c1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -24,6 +24,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
@@ -51,6 +53,9 @@ import org.apache.carbondata.processing.newflow.dictionary.PreCreatedDictionary;
  */
 public class PrimitiveDataType implements GenericDataType<Object> {
 
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(PrimitiveDataType.class.getName());
+
   /**
    * surrogate index
    */
@@ -113,7 +118,7 @@ public class PrimitiveDataType implements GenericDataType<Object> {
   public PrimitiveDataType(String name, String parentname, String columnId,
       CarbonDimension carbonDimension, Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, boolean tableInitialize, Map<Object, Integer> localCache) {
+      String storePath, Map<Object, Integer> localCache) {
     this.name = name;
     this.parentname = parentname;
     this.columnId = columnId;
@@ -134,13 +139,9 @@ public class PrimitiveDataType implements GenericDataType<Object> {
           }
           DictionaryMessage dictionaryMessage = new DictionaryMessage();
           dictionaryMessage.setColumnName(carbonDimension.getColName());
-          dictionaryMessage.setTableUniqueName(carbonTableIdentifier.getTableUniqueName());
           // for table initialization
-          dictionaryMessage.setType(DictionaryMessageType.TABLE_INTIALIZATION);
+          dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
           dictionaryMessage.setData("0");
-          if (tableInitialize) {
-            client.getDictionary(dictionaryMessage);
-          }
           // for generate dictionary
           dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
           dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
index 2614e17..8d4d5a3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cache;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
@@ -45,6 +47,9 @@ import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
 
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DictionaryFieldConverterImpl.class.getName());
+
   private BiDictionary<Integer, Object> dictionaryGenerator;
 
   private int index;
@@ -62,7 +67,7 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
   public DictionaryFieldConverterImpl(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
-      DictionaryClient client, boolean useOnePass, String storePath, boolean tableInitialize,
+      DictionaryClient client, boolean useOnePass, String storePath,
       Map<Object, Integer> localCache, boolean isEmptyBadRecord) throws IOException {
     this.index = index;
     this.carbonDimension = (CarbonDimension) dataField.getColumn();
@@ -80,13 +85,9 @@ public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConvert
       }
       dictionaryMessage = new DictionaryMessage();
       dictionaryMessage.setColumnName(dataField.getColumn().getColName());
-      dictionaryMessage.setTableUniqueName(carbonTableIdentifier.getTableUniqueName());
       // for table initialization
-      dictionaryMessage.setType(DictionaryMessageType.TABLE_INTIALIZATION);
+      dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
       dictionaryMessage.setData("0");
-      if (tableInitialize) {
-        client.getDictionary(dictionaryMessage);
-      }
       // for generate dictionary
       dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
       dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
index e9b0a78..1aada19 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
@@ -63,7 +63,7 @@ public class FieldEncoderFactory {
   public FieldConverter createFieldEncoder(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, String storePath, boolean tableInitialize,
+      DictionaryClient client, Boolean useOnePass, String storePath,
       Map<Object, Integer> localCache, boolean isEmptyBadRecord)
       throws IOException {
     // Converters are only needed for dimensions and measures it return null.
@@ -75,11 +75,11 @@ public class FieldEncoderFactory {
       } else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
           !dataField.getColumn().isComplex()) {
         return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, nullFormat,
-            index, client, useOnePass, storePath, tableInitialize, localCache, isEmptyBadRecord);
+            index, client, useOnePass, storePath, localCache, isEmptyBadRecord);
       } else if (dataField.getColumn().isComplex()) {
         return new ComplexFieldConverterImpl(
             createComplexType(dataField, cache, carbonTableIdentifier,
-                    client, useOnePass, storePath, tableInitialize, localCache), index);
+                    client, useOnePass, storePath, localCache), index);
       } else {
         return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
       }
@@ -94,9 +94,9 @@ public class FieldEncoderFactory {
   private static GenericDataType createComplexType(DataField dataField,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, boolean tableInitialize, Map<Object, Integer> localCache) {
+      String storePath, Map<Object, Integer> localCache) {
     return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(), cache,
-        carbonTableIdentifier, client, useOnePass, storePath, tableInitialize, localCache);
+        carbonTableIdentifier, client, useOnePass, storePath, localCache);
   }
 
   /**
@@ -107,7 +107,7 @@ public class FieldEncoderFactory {
   private static GenericDataType createComplexType(CarbonColumn carbonColumn, String parentName,
       Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
       CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, boolean tableInitialize, Map<Object, Integer> localCache) {
+      String storePath, Map<Object, Integer> localCache) {
     switch (carbonColumn.getDataType()) {
       case ARRAY:
         List<CarbonDimension> listOfChildDimensions =
@@ -117,7 +117,7 @@ public class FieldEncoderFactory {
             new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
         for (CarbonDimension dimension : listOfChildDimensions) {
           arrayDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, tableInitialize, localCache));
+              carbonTableIdentifier, client, useOnePass, storePath, localCache));
         }
         return arrayDataType;
       case STRUCT:
@@ -128,7 +128,7 @@ public class FieldEncoderFactory {
             new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
         for (CarbonDimension dimension : dimensions) {
           structDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, tableInitialize, localCache));
+              carbonTableIdentifier, client, useOnePass, storePath, localCache));
         }
         return structDataType;
       case MAP:
@@ -136,7 +136,7 @@ public class FieldEncoderFactory {
       default:
         return new PrimitiveDataType(carbonColumn.getColName(), parentName,
             carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache,
-            carbonTableIdentifier, client, useOnePass, storePath, tableInitialize, localCache);
+            carbonTableIdentifier, client, useOnePass, storePath, localCache);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ee5f65f7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
index 3d323e0..eecb0e9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
@@ -104,7 +104,7 @@ public class RowConverterImpl implements RowConverter {
           .createFieldEncoder(fields[i], cache,
               configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
               configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
-              true, localCaches[i], isEmptyBadRecord);
+              localCaches[i], isEmptyBadRecord);
       fieldConverterList.add(fieldConverter);
     }
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
@@ -209,7 +209,7 @@ public class RowConverterImpl implements RowConverter {
       try {
         fieldConverter = FieldEncoderFactory.getInstance().createFieldEncoder(fields[i], cache,
             configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
-            configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(), false,
+            configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
             localCaches[i], isEmptyBadRecord);
       } catch (IOException e) {
         throw new RuntimeException(e);


Mime
View raw message