You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ri...@apache.org on 2021/07/29 15:42:59 UTC

[phoenix] 01/02: PHOENIX-6454: Add feature to SchemaTool to get the DDL in specification mode (#1229)

This is an automated email from the ASF dual-hosted git repository.

richardantal pushed a commit to branch 5.1
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit cae91b89cab8f80893d89deeca86f6c14e3565ef
Author: Swaroopa Kadam <sw...@gmail.com>
AuthorDate: Fri May 14 09:47:35 2021 -0700

    PHOENIX-6454: Add feature to SchemaTool to get the DDL in specification mode (#1229)
    
    Co-authored-by: Swaroopa Kadam <s....@apache.org>
---
 .../phoenix/compile/CreateTableCompiler.java       |   2 +-
 .../apache/phoenix/parse/CreateIndexStatement.java |  13 ++
 .../apache/phoenix/parse/CreateTableStatement.java |  33 ++-
 .../apache/phoenix/parse/IndexKeyConstraint.java   |  16 ++
 .../apache/phoenix/parse/PrimaryKeyConstraint.java |  18 +-
 phoenix-tools/pom.xml                              |   5 +
 ...tionToolIT.java => SchemaToolExtractionIT.java} |  18 +-
 .../phoenix/schema/SchemaToolSynthesisIT.java      | 205 ++++++++++++++++++
 .../it/resources/synthesis/alter_add_property.sql  |  34 +--
 .../resources/synthesis/alter_change_property.sql  |  34 +--
 .../synthesis/alter_index_add_property.sql         |  27 +--
 .../synthesis/alter_index_change_property.sql      |  27 +--
 .../src/it/resources/synthesis/alter_table_add.sql |  34 +--
 .../it/resources/synthesis/alter_table_add_pk.sql  |   8 +
 .../it/resources/synthesis/alter_table_drop.sql    |  34 +--
 .../resources/synthesis/alter_table_multiple.sql   |  34 ++-
 .../src/it/resources/synthesis/alter_view_add.sql  |  39 ++--
 .../src/it/resources/synthesis/alter_view_drop.sql |  39 ++--
 .../src/it/resources/synthesis/create_function.sql |  25 +--
 .../src/it/resources/synthesis/drop_index.sql      |  27 +--
 .../src/it/resources/synthesis/drop_table.sql      |  34 ++-
 .../resources/synthesis/mismatched_entity_name.sql |  34 +--
 .../phoenix/schema/SchemaExtractionProcessor.java  |   3 +-
 .../org/apache/phoenix/schema/SchemaProcessor.java |  27 +--
 .../org/apache/phoenix/schema/SchemaSQLUtil.java   | 168 +++++++++++++++
 .../phoenix/schema/SchemaSynthesisProcessor.java   | 235 +++++++++++++++++++++
 .../{SchemaExtractionTool.java => SchemaTool.java} |  57 +++--
 27 files changed, 884 insertions(+), 346 deletions(-)

diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index ec31a52..c8933f3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -117,7 +117,7 @@ public class CreateTableCompiler {
             }
         }
         if (overideColumnDefs != null) {
-            create = new CreateTableStatement (create,overideColumnDefs);
+            create = new CreateTableStatement(create,overideColumnDefs);
         }
         final CreateTableStatement finalCreate = create;
         
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
index 33f05e9..172b95e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateIndexStatement.java
@@ -54,6 +54,19 @@ public class CreateIndexStatement extends SingleTableStatement {
         this.udfParseNodes = udfParseNodes;
     }
 
+    public CreateIndexStatement(CreateIndexStatement createStmt, ListMultimap<String, Pair<String, Object>> finalProps) {
+        super(createStmt.getTable(), createStmt.getBindCount());
+        this.indexTableName = createStmt.getIndexTableName();
+        this.indexKeyConstraint = createStmt.getIndexConstraint();
+        this.includeColumns = createStmt.getIncludeColumns();
+        this.splitNodes = createStmt.getSplitNodes();
+        this.props = finalProps;
+        this.ifNotExists = createStmt.ifNotExists();
+        this.indexType = createStmt.getIndexType();
+        this.async = createStmt.isAsync();
+        this.udfParseNodes = createStmt.getUdfParseNodes();
+    }
+
     public IndexKeyConstraint getIndexConstraint() {
         return indexKeyConstraint;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
index ebd0a07..fcde8f1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateTableStatement.java
@@ -55,7 +55,36 @@ public class CreateTableStatement extends MutableStatement {
         this.whereClause = createTable.whereClause;
         this.immutableRows = createTable.immutableRows;
     }
-    
+
+    public CreateTableStatement(CreateTableStatement createTable, PrimaryKeyConstraint pkConstraint,
+            List<ColumnDef> columns) {
+        this.tableName = createTable.tableName;
+        this.tableType = createTable.tableType;
+        this.columns = ImmutableList.copyOf(columns);
+        this.pkConstraint = pkConstraint;
+        this.splitNodes = createTable.splitNodes;
+        this.bindCount = createTable.bindCount;
+        this.props = createTable.props;
+        this.ifNotExists = createTable.ifNotExists;
+        this.baseTableName = createTable.baseTableName;
+        this.whereClause = createTable.whereClause;
+        this.immutableRows = createTable.immutableRows;
+    }
+
+    public CreateTableStatement(CreateTableStatement createTable, ListMultimap<String,Pair<String,Object>>  props, List<ColumnDef> columns) {
+        this.tableName = createTable.tableName;
+        this.tableType = createTable.tableType;
+        this.columns = ImmutableList.copyOf(columns);
+        this.pkConstraint = createTable.pkConstraint;
+        this.splitNodes = createTable.splitNodes;
+        this.bindCount = createTable.bindCount;
+        this.props = props;
+        this.ifNotExists = createTable.ifNotExists;
+        this.baseTableName = createTable.baseTableName;
+        this.whereClause = createTable.whereClause;
+        this.immutableRows = createTable.immutableRows;
+    }
+
     protected CreateTableStatement(TableName tableName, ListMultimap<String,Pair<String,Object>> props, List<ColumnDef> columns, PrimaryKeyConstraint pkConstraint,
             List<ParseNode> splitNodes, PTableType tableType, boolean ifNotExists, 
             TableName baseTableName, ParseNode whereClause, int bindCount, Boolean immutableRows) {
@@ -71,7 +100,7 @@ public class CreateTableStatement extends MutableStatement {
         this.whereClause = whereClause;
         this.immutableRows = immutableRows;
     }
-    
+
     public ParseNode getWhereClause() {
         return whereClause;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
index dfa9a1a..ed9e04c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.parse;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.hbase.util.Pair;
 
@@ -38,4 +39,19 @@ public class IndexKeyConstraint {
     public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
         return columnNameToSortOrder;
     }
+
+    @Override
+    public String toString() {
+        StringBuffer sb = new StringBuffer();
+        for(Pair<ParseNode, SortOrder> entry : columnNameToSortOrder) {
+            if(sb.length()!=0) {
+                sb.append(", ");
+            }
+            sb.append(entry.getFirst().toString());
+            if(entry.getSecond() != SortOrder.getDefault()) {
+                sb.append(" "+entry.getSecond());
+            }
+        }
+        return sb.toString();
+    }
 }
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
index bf978a9..e6a65a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/PrimaryKeyConstraint.java
@@ -36,7 +36,7 @@ public class PrimaryKeyConstraint extends NamedNode {
     private final Map<ColumnName, Pair<ColumnName, Boolean>> columnNameToRowTimestamp;
     private final int numColumnsWithRowTimestamp;
     
-    PrimaryKeyConstraint(String name, List<ColumnDefInPkConstraint> columnDefs) {
+    public PrimaryKeyConstraint(String name, List<ColumnDefInPkConstraint> columnDefs) {
         super(name);
         if (columnDefs == null) {
             this.columns = Collections.<Pair<ColumnName, SortOrder>>emptyList();
@@ -91,5 +91,19 @@ public class PrimaryKeyConstraint extends NamedNode {
     public boolean equals(Object obj) {
         return super.equals(obj);
     }
-    
+
+    @Override
+    public String toString() {
+        StringBuffer sb = new StringBuffer();
+        for(Map.Entry<ColumnName, Pair<ColumnName, SortOrder>> entry : columnNameToSortOrder.entrySet()) {
+            if(sb.length()!=0) {
+                sb.append(", ");
+            }
+            sb.append(entry.getKey());
+            if(entry.getValue().getSecond() != SortOrder.getDefault()) {
+                sb.append(" "+entry.getValue().getSecond());
+            }
+        }
+        return sb.toString();
+    }
 }
diff --git a/phoenix-tools/pom.xml b/phoenix-tools/pom.xml
index 8a106c2..47a41d1 100644
--- a/phoenix-tools/pom.xml
+++ b/phoenix-tools/pom.xml
@@ -55,6 +55,11 @@
             <groupId>org.apache.phoenix.thirdparty</groupId>
             <artifactId>phoenix-shaded-commons-cli</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.phoenix.thirdparty</groupId>
+            <artifactId>phoenix-shaded-guava</artifactId>
+        </dependency>
+
 
         <dependency>
             <groupId>org.apache.phoenix</groupId>
diff --git a/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaExtractionToolIT.java b/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolExtractionIT.java
similarity index 97%
rename from phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaExtractionToolIT.java
rename to phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolExtractionIT.java
index 34b09db..ff19873 100644
--- a/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaExtractionToolIT.java
+++ b/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolExtractionIT.java
@@ -46,7 +46,7 @@ import java.util.Properties;
 import static junit.framework.TestCase.fail;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 
-public class SchemaExtractionToolIT extends ParallelStatsEnabledIT {
+public class SchemaToolExtractionIT extends ParallelStatsEnabledIT {
 
     @BeforeClass
     public static synchronized void setup() throws Exception {
@@ -479,14 +479,14 @@ public class SchemaExtractionToolIT extends ParallelStatsEnabledIT {
         if (tenantId == null) {
             try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
                 executeCreateStatements(conn, queries);
-                String [] args = {"-tb", tableName, "-s", schemaName};
-                output = extractSchema(conn, args);
+                String [] args = {"-m", "EXTRACT", "-tb", tableName, "-s", schemaName};
+                output = runSchemaTool(conn, args);
             }
         } else {
             try (Connection conn = getTenantConnection(getUrl(), tenantId)) {
                 executeCreateStatements(conn, queries);
-                String [] args = {"-tb", tableName, "-s", schemaName, "-t", tenantId};
-                output = extractSchema(conn, args);
+                String [] args = {"-m", "EXTRACT","-tb", tableName, "-s", schemaName, "-t", tenantId};
+                output = runSchemaTool(conn, args);
             }
         }
         return output;
@@ -499,9 +499,11 @@ public class SchemaExtractionToolIT extends ParallelStatsEnabledIT {
         conn.commit();
     }
 
-    private String extractSchema(Connection conn, String [] args) throws Exception {
-        SchemaExtractionTool set = new SchemaExtractionTool();
-        set.setConf(conn.unwrap(PhoenixConnection.class).getQueryServices().getConfiguration());
+    public static String runSchemaTool(Connection conn, String [] args) throws Exception {
+        SchemaTool set = new SchemaTool();
+        if(conn!=null) {
+            set.setConf(conn.unwrap(PhoenixConnection.class).getQueryServices().getConfiguration());
+        }
         set.run(args);
         return set.getOutput();
     }
diff --git a/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolSynthesisIT.java b/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolSynthesisIT.java
new file mode 100644
index 0000000..8ac0774
--- /dev/null
+++ b/phoenix-tools/src/it/java/org/apache/phoenix/schema/SchemaToolSynthesisIT.java
@@ -0,0 +1,205 @@
+/*
+ * 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.phoenix.schema;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.File;
+import java.net.URL;
+
+import static org.apache.phoenix.schema.SchemaSynthesisProcessor.ENTITY_NAME_IN_BASE_AND_ALTER_DDL_DON_T_MATCH;
+import static org.apache.phoenix.schema.SchemaSynthesisProcessor.UNSUPPORTED_DDL_EXCEPTION;
+import static org.apache.phoenix.schema.SchemaToolExtractionIT.runSchemaTool;
+
+public class SchemaToolSynthesisIT {
+
+    private static final String SYNTHESIS_DIR = "synthesis/";
+    URL fileUrl = SchemaToolSynthesisIT.class.getClassLoader().getResource(
+            SYNTHESIS_DIR);
+    String filePath = new File(fileUrl.getFile()).getAbsolutePath();
+
+
+    @Test
+    // Adding new column RELATED_COMMAND
+    public void testCreateTableStatement_addColumn() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE\n"
+                + "(ORG_ID CHAR(15) NOT NULL,\n" + "SOME_ANOTHER_ID BIGINT NOT NULL,\n"
+                + "TYPE VARCHAR,\n" + "STATUS VARCHAR,\n" + "START_TIMESTAMP BIGINT,\n"
+                + "END_TIMESTAMP BIGINT,\n" + "PARAMS VARCHAR,\n" + "RESULT VARCHAR,\n"
+                + "RELATED_COMMAND BIGINT\n"
+                + "CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID))\n"
+                + "VERSIONS=1,MULTI_TENANT=false,REPLICATION_SCOPE=1,TTL=31536000";
+        String baseDDL = filePath+"/alter_table_add.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Dropping TYPE column
+    public void testCreateTableStatement_dropColumn() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE\n"
+                + "(ORG_ID CHAR(15) NOT NULL,\n" + "SOME_ANOTHER_ID BIGINT NOT NULL,\n"
+                + "STATUS VARCHAR,\n" + "START_TIMESTAMP BIGINT,\n" + "END_TIMESTAMP BIGINT,\n"
+                + "PARAMS VARCHAR,\n" + "RESULT VARCHAR\n"
+                + "CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID))\n"
+                + "VERSIONS=1,MULTI_TENANT=false,REPLICATION_SCOPE=1,TTL=31536000";
+        String baseDDL = filePath+"/alter_table_drop.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Changing REPLICATION SCOPE from 1 to 0
+    public void testCreateTableStatement_changeProperty() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE\n"
+                + "(ORG_ID CHAR(15) NOT NULL,\n" + "SOME_ANOTHER_ID BIGINT NOT NULL,\n"
+                + "TYPE VARCHAR,\n" + "STATUS VARCHAR,\n" + "START_TIMESTAMP BIGINT,\n"
+                + "END_TIMESTAMP BIGINT,\n" + "PARAMS VARCHAR,\n" + "RESULT VARCHAR\n"
+                + "CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID))\n"
+                + "MULTI_TENANT=false,REPLICATION_SCOPE=0,TTL=31536000,VERSIONS=1";
+        String baseDDL = filePath+"/alter_change_property.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Adding DISABLE_MIGRATION property
+    public void testCreateTableStatement_addProperty() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE\n"
+                + "(ORG_ID CHAR(15) NOT NULL,\n" + "SOME_ANOTHER_ID BIGINT NOT NULL,\n"
+                + "TYPE VARCHAR,\n" + "STATUS VARCHAR,\n" + "START_TIMESTAMP BIGINT,\n"
+                + "END_TIMESTAMP BIGINT,\n" + "PARAMS VARCHAR,\n" + "RESULT VARCHAR\n"
+                + "CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID))\n"
+                + "DISABLE_MIGRATION=true,MULTI_TENANT=false,REPLICATION_SCOPE=1,TTL=31536000,VERSIONS=1";
+        String baseDDL = filePath+"/alter_add_property.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Add NEW_COLUMN to the view
+    public void testCreateViewStatement_addColumn() throws Exception {
+        String expected = "CREATE VIEW IF NOT EXISTS TEST.SAMPLE_VIEW\n"
+                + "(DATE_TIME1 DATE NOT NULL,\n" + "INT1 BIGINT NOT NULL,\n"
+                + "SOME_ID CHAR(15) NOT NULL,\n" + "DOUBLE1 DECIMAL(12,3),\n"
+                + "IS_BOOLEAN BOOLEAN,\n" + "RELATE CHAR(15),\n" + "TEXT1 VARCHAR,\n"
+                + "TEXT_READ_ONLY VARCHAR,\n" + "NEW_COLUMN VARCHAR(20)\n"
+                + "CONSTRAINT PKVIEW PRIMARY KEY (DATE_TIME1 DESC, SOME_ID, INT1))\n"
+                + "AS SELECT * FROM TEST.SAMPLE_TABLE_VIEW WHERE FILTER_PREFIX = 'abc'";
+        String baseDDL = filePath+"/alter_view_add.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Add SOME_ID VARCHAR NULL to the PK of the table
+    public void testCreateTableStatement_addColumn_pk() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.TABLE_1\n" + "(STATE CHAR(1) NOT NULL,\n"
+                + "SOME_ID VARCHAR\n" + "CONSTRAINT PK PRIMARY KEY (STATE, SOME_ID))";
+        String baseDDL = filePath+"/alter_table_add_pk.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Dropping DOUBLE1 column from the view
+    public void testCreateViewStatement_dropColumn() throws Exception {
+        String expected = "CREATE VIEW IF NOT EXISTS TEST.SAMPLE_VIEW\n"
+                + "(DATE_TIME1 DATE NOT NULL,\n" + "INT1 BIGINT NOT NULL,\n"
+                + "SOME_ID CHAR(15) NOT NULL,\n" + "IS_BOOLEAN BOOLEAN,\n" + "RELATE CHAR(15),\n"
+                + "TEXT1 VARCHAR,\n" + "TEXT_READ_ONLY VARCHAR\n"
+                + "CONSTRAINT PKVIEW PRIMARY KEY (DATE_TIME1 DESC, SOME_ID, INT1))\n"
+                + "AS SELECT * FROM TEST.SAMPLE_TABLE_VIEW WHERE FILTER_PREFIX = 'abc'";
+        String baseDDL = filePath+"/alter_view_drop.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // changing TTL
+    public void testCreateIndexStatement_changeProperty() throws Exception {
+        String expected = "CREATE INDEX IF NOT EXISTS ANOTHER_INDEX_ON_SOME_TABLE\n"
+                + "ON TEST.SOME_TABLE_WITH_INDEX(SOME_VALUE_COL_1, SOME_VALUE_COL)\n"
+                + "INCLUDE (TEXT_VALUE) ASYNC\n" + "TTL=5000";
+        String baseDDL = filePath+"/alter_index_change_property.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // Adding REPLICATION_SCOPE to 1
+    public void testCreateIndexStatement_addProperty() throws Exception {
+        String expected = "CREATE INDEX IF NOT EXISTS ANOTHER_INDEX_ON_SOME_TABLE\n"
+                + "ON TEST.SOME_TABLE_WITH_INDEX(SOME_VALUE_COL_1, SOME_VALUE_COL)\n"
+                + "INCLUDE (TEXT_VALUE) ASYNC\n" + "TTL=123000,REPLICATION_SCOPE=1";
+        String baseDDL = filePath+"/alter_index_add_property.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // drop table
+    public void testCreateTableStatement_dropTable() throws Exception {
+        String expected = "";
+        String baseDDL = filePath+"/drop_table.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // drop table
+    public void testCreateIndexStatement_dropIndex() throws Exception {
+        String expected = "";
+        String baseDDL = filePath+"/drop_index.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    private void runAndVerify(String expected, String baseDDL) throws Exception {
+        String[] arg = { "-m", "SYNTH", "-d", baseDDL };
+        String result = runSchemaTool(null, arg);
+        Assert.assertEquals(expected, result);
+    }
+
+    @Test
+    // Alter DDL file can have multiple alter statements
+    public void testMultipleAlterDDL() throws Exception {
+        String expected = "CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE\n"
+                + "(ORG_ID CHAR(15) NOT NULL,\n" + "SOME_ANOTHER_ID BIGINT NOT NULL,\n"
+                + "TYPE VARCHAR,\n" + "STATUS VARCHAR,\n" + "START_TIMESTAMP BIGINT,\n"
+                + "END_TIMESTAMP BIGINT,\n" + "PARAMS VARCHAR,\n" + "RESULT VARCHAR,\n"
+                + "SOME_NEW_COLUMN BIGINT\n"
+                + "CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID))\n"
+                + "MULTI_TENANT=false,REPLICATION_SCOPE=1,TTL=2000,VERSIONS=1";
+        String baseDDL = filePath+"/alter_table_multiple.sql";
+        runAndVerify(expected, baseDDL);
+    }
+
+    @Test
+    // create DDL and alter DDL should be for the same entity
+    public void testMismatchedEntityNames() throws Exception {
+        String baseDDL = filePath+"/mismatched_entity_name.sql";
+        String [] arg = {"-m", "SYNTH", "-d", baseDDL};
+        try {
+            runSchemaTool(null, arg);
+        } catch (Exception e) {
+            e.getMessage().equalsIgnoreCase(ENTITY_NAME_IN_BASE_AND_ALTER_DDL_DON_T_MATCH);
+        }
+    }
+
+    @Test
+    public void testUnsupportedStatements() {
+        String baseDDL = filePath+"/create_function.sql";
+        String [] arg = {"-m", "SYNTH", "-d", baseDDL};
+        try {
+            runSchemaTool(null, arg);
+        } catch (Exception e) {
+            e.getMessage().equalsIgnoreCase(UNSUPPORTED_DDL_EXCEPTION);
+        }
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_add_property.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_add_property.sql
index dfa9a1a..bb436f7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_add_property.sql
@@ -15,27 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.SAMPLE_TABLE SET DISABLE_MIGRATION=true;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_change_property.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_change_property.sql
index dfa9a1a..79d42a2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_change_property.sql
@@ -15,27 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.SAMPLE_TABLE SET REPLICATION_SCOPE=0;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_index_add_property.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_index_add_property.sql
index dfa9a1a..2d32a66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_index_add_property.sql
@@ -15,27 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE INDEX IF NOT EXISTS ANOTHER_INDEX_ON_SOME_TABLE
+ON TEST.SOME_TABLE_WITH_INDEX (SOME_VALUE_COL_1, SOME_VALUE_COL)
+INCLUDE (TEXT_VALUE) ASYNC TTL=123000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.ANOTHER_INDEX_ON_SOME_TABLE SET REPLICATION_SCOPE=1;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_index_change_property.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_index_change_property.sql
index dfa9a1a..16c4150 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_index_change_property.sql
@@ -15,27 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE INDEX IF NOT EXISTS ANOTHER_INDEX_ON_SOME_TABLE
+ON TEST.SOME_TABLE_WITH_INDEX (SOME_VALUE_COL_1, SOME_VALUE_COL)
+INCLUDE (TEXT_VALUE) ASYNC TTL=123000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.ANOTHER_INDEX_ON_SOME_TABLE SET TTL=5000;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_table_add.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_table_add.sql
index dfa9a1a..a22170f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_table_add.sql
@@ -15,27 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.SAMPLE_TABLE ADD IF NOT EXISTS RELATED_COMMAND BIGINT NULL;
\ No newline at end of file
diff --git a/phoenix-tools/src/it/resources/synthesis/alter_table_add_pk.sql b/phoenix-tools/src/it/resources/synthesis/alter_table_add_pk.sql
new file mode 100644
index 0000000..6835cf0
--- /dev/null
+++ b/phoenix-tools/src/it/resources/synthesis/alter_table_add_pk.sql
@@ -0,0 +1,8 @@
+CREATE TABLE IF NOT EXISTS TEST.TABLE_1 (
+	STATE CHAR(1) NOT NULL,
+	CONSTRAINT PK PRIMARY KEY
+    (
+       STATE
+    )
+);
+ALTER TABLE TEST.TABLE_1 ADD IF NOT EXISTS SOME_ID VARCHAR NULL PRIMARY KEY;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_table_drop.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_table_drop.sql
index dfa9a1a..587adb8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_table_drop.sql
@@ -15,27 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.SAMPLE_TABLE DROP COLUMN TYPE;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_table_multiple.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_table_multiple.sql
index dfa9a1a..4734722 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_table_multiple.sql
@@ -15,27 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
+ALTER TABLE TEST.SAMPLE_TABLE SET TTL=2000;
 
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER TABLE TEST.SAMPLE_TABLE ADD IF NOT EXISTS SOME_NEW_COLUMN BIGINT NULL;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_view_add.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_view_add.sql
index dfa9a1a..cfc1654 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_view_add.sql
@@ -15,27 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE VIEW IF NOT EXISTS TEST.SAMPLE_VIEW  (
+     DATE_TIME1 DATE NOT NULL,
+     INT1 BIGINT NOT NULL,
+     SOME_ID CHAR(15) NOT NULL,
+     DOUBLE1 DECIMAL(12, 3),
+     IS_BOOLEAN BOOLEAN,
+     RELATE CHAR(15),
+     TEXT1 VARCHAR,
+     TEXT_READ_ONLY VARCHAR
+     CONSTRAINT PKVIEW PRIMARY KEY
+     (
+         DATE_TIME1 DESC, INT1, SOME_ID
+     )
+ )
+AS SELECT * FROM TEST.SAMPLE_TABLE_VIEW WHERE FILTER_PREFIX = 'abc';
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER VIEW TEST.SAMPLE_VIEW ADD NEW_COLUMN VARCHAR(20);
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/alter_view_drop.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/alter_view_drop.sql
index dfa9a1a..c0403e3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/alter_view_drop.sql
@@ -15,27 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE VIEW IF NOT EXISTS TEST.SAMPLE_VIEW  (
+     DATE_TIME1 DATE NOT NULL,
+     INT1 BIGINT NOT NULL,
+     SOME_ID CHAR(15) NOT NULL,
+     DOUBLE1 DECIMAL(12, 3),
+     IS_BOOLEAN BOOLEAN,
+     RELATE CHAR(15),
+     TEXT1 VARCHAR,
+     TEXT_READ_ONLY VARCHAR
+     CONSTRAINT PKVIEW PRIMARY KEY
+     (
+         DATE_TIME1 DESC, INT1, SOME_ID
+     )
+ )
+AS SELECT * FROM TEST.SAMPLE_TABLE_VIEW WHERE FILTER_PREFIX = 'abc';
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER VIEW TEST.SAMPLE_VIEW DROP COLUMN DOUBLE1;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/create_function.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/create_function.sql
index dfa9a1a..39131bd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/create_function.sql
@@ -15,27 +15,4 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ CREATE FUNCTION my_reverse(varchar) returns varchar as 'com.mypackage.MyReverseFunction'
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/drop_index.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/drop_index.sql
index dfa9a1a..5331606 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/drop_index.sql
@@ -15,27 +15,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE INDEX IF NOT EXISTS ANOTHER_INDEX_ON_SOME_TABLE
+ON TEST.SOME_TABLE_WITH_INDEX (SOME_VALUE_COL_1, SOME_VALUE_COL)
+INCLUDE (TEXT_VALUE) ASYNC TTL=123000;
 
-import java.util.Collections;
-import java.util.List;
+ALTER TABLE TEST.ANOTHER_INDEX_ON_SOME_TABLE SET REPLICATION_SCOPE=1;
 
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+DROP INDEX ANOTHER_INDEX_ON_SOME_TABLE ON TEST.SOME_TABLE_WITH_INDEX;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/drop_table.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/drop_table.sql
index dfa9a1a..e19f35f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/drop_table.sql
@@ -15,27 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
+ALTER TABLE TEST.SAMPLE_TABLE ADD IF NOT EXISTS RELATED_COMMAND BIGINT NULL;
 
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+DROP TABLE TEST.SAMPLE_TABLE CASCADE;
\ No newline at end of file
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/it/resources/synthesis/mismatched_entity_name.sql
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/it/resources/synthesis/mismatched_entity_name.sql
index dfa9a1a..fc97d42 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/it/resources/synthesis/mismatched_entity_name.sql
@@ -15,27 +15,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+CREATE TABLE IF NOT EXISTS TEST.SAMPLE_TABLE (
+   ORG_ID CHAR(15) NOT NULL,
+   SOME_ANOTHER_ID BIGINT NOT NULL,
+   TYPE VARCHAR,
+   STATUS VARCHAR,
+   START_TIMESTAMP BIGINT,
+   END_TIMESTAMP BIGINT,
+   PARAMS VARCHAR,   RESULT VARCHAR
+   CONSTRAINT PK PRIMARY KEY (ORG_ID, SOME_ANOTHER_ID)
+) VERSIONS=1,MULTI_TENANT=FALSE,REPLICATION_SCOPE=1,TTL=31536000;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+ALTER VIEW TEST.SAMPLE_VIEW ADD NEW_COLUMN VARCHAR(20);
\ No newline at end of file
diff --git a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionProcessor.java b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionProcessor.java
index 89d1b46..af16ba0 100644
--- a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionProcessor.java
+++ b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionProcessor.java
@@ -46,7 +46,7 @@ import java.util.ArrayList;
 
 import static org.apache.phoenix.util.MetaDataUtil.SYNCED_DATA_TABLE_AND_INDEX_COL_FAM_PROPERTIES;
 
-public class SchemaExtractionProcessor {
+public class SchemaExtractionProcessor implements SchemaProcessor {
     Map<String, String> defaultProps = new HashMap<>();
     Map<String, String> definedProps = new HashMap<>();
 
@@ -67,6 +67,7 @@ public class SchemaExtractionProcessor {
         this.table = getPTable(pSchemaName, pTableName);
     }
 
+    @Override
     public String process() throws Exception {
         if (ddl != null) {
             return ddl;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaProcessor.java
similarity index 50%
copy from phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
copy to phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaProcessor.java
index dfa9a1a..f0bc8ae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/IndexKeyConstraint.java
+++ b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaProcessor.java
@@ -15,27 +15,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.parse;
+package org.apache.phoenix.schema;
 
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.hbase.util.Pair;
-
-import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList;
-
-import org.apache.phoenix.schema.SortOrder;
-
-public class IndexKeyConstraint {
-	public static final IndexKeyConstraint EMPTY = new IndexKeyConstraint(Collections.<Pair<ParseNode, SortOrder>>emptyList());
-
-    private final List<Pair<ParseNode, SortOrder>> columnNameToSortOrder;
-    
-    IndexKeyConstraint(List<Pair<ParseNode, SortOrder>> parseNodeAndSortOrder) {
-        this.columnNameToSortOrder = ImmutableList.copyOf(parseNodeAndSortOrder);
-    }
-
-    public List<Pair<ParseNode, SortOrder>> getParseNodeAndSortOrderList() {
-        return columnNameToSortOrder;
-    }
-}
\ No newline at end of file
+public interface SchemaProcessor {
+    String process() throws Exception;
+}
diff --git a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSQLUtil.java b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSQLUtil.java
new file mode 100644
index 0000000..c565b51
--- /dev/null
+++ b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSQLUtil.java
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema;
+
+import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.CreateIndexStatement;
+import org.apache.phoenix.parse.CreateTableStatement;
+
+import java.util.List;
+import java.util.Map;
+
+public class SchemaSQLUtil {
+
+    protected static String getCreateTableSQL(CreateTableStatement createStmt) {
+        if (createStmt == null) {
+            return "";
+        }
+        StringBuffer sb = new StringBuffer()
+                .append("CREATE "+createStmt.getTableType() + " ");
+        if (createStmt.ifNotExists()) {
+            sb.append("IF NOT EXISTS ");
+        }
+        sb.append(createStmt.getTableName()).append("\n")
+                .append(getColumnDefListToString(createStmt))
+                .append("\nCONSTRAINT "+createStmt.getPrimaryKeyConstraint().getName()+" PRIMARY KEY")
+                .append(" ("+createStmt.getPrimaryKeyConstraint().toString()+"))"
+                        .replaceAll(",", ",\n"));
+        if (createStmt.getTableType().equals(PTableType.VIEW)) {
+            sb.append("\nAS SELECT * FROM " + createStmt.getBaseTableName());
+            if (createStmt.getWhereClause()!=null) {
+                sb.append(" WHERE " +createStmt.getWhereClause());
+            }
+        }
+        appendProperties(sb, createStmt.getProps());
+        return sb.toString();
+    }
+
+    protected static String getCreateIndexSQL(CreateIndexStatement createStmt) {
+        if (createStmt == null) {
+            return "";
+        }
+        StringBuffer sb = new StringBuffer()
+                .append("CREATE"
+                        + (createStmt.getIndexType().equals(PTable.IndexType.LOCAL) ? " "+createStmt.getIndexType() : "")
+                        + " INDEX ");
+        if (createStmt.ifNotExists()) {
+            sb.append("IF NOT EXISTS ");
+        }
+        sb.append(createStmt.getIndexTableName().getTableName()).append("\n")
+                .append("ON "+createStmt.getTable().getName())
+                .append("("+createStmt.getIndexConstraint().toString()).append(")");
+        if (createStmt.getIncludeColumns()!=null && !createStmt.getIncludeColumns().isEmpty()) {
+            sb.append("\nINCLUDE ");
+            sb.append(getColumnListToString(createStmt.getIncludeColumns()));
+        }
+        if (createStmt.isAsync()) {
+            sb.append(" ASYNC");
+        }
+        appendProperties(sb, createStmt.getProps());
+        return sb.toString();
+    }
+
+    private static String getColumnListToString(List<ColumnName> columnNames) {
+        StringBuffer sb = new StringBuffer();
+        for(ColumnName cName : columnNames) {
+            if (sb.length()==0) {
+                sb.append("(");
+            }
+            sb.append(cName.toString()).append(",\n");
+        }
+        if (sb.length()!=0) {
+            sb.deleteCharAt(sb.length()-1).deleteCharAt(sb.length()-1);
+            sb.append(")");
+        }
+        return sb.toString();
+    }
+
+    private static String getColumnDefListToString(CreateTableStatement createStatement) {
+        List<ColumnDef> colDef = createStatement.getColumnDefs();
+        StringBuffer sb = new StringBuffer();
+        for(ColumnDef cDef : colDef) {
+            String columnString = getColumnInfoString(cDef);
+            if (sb.length()==0) {
+                sb.append("(");
+            } else {
+                sb.append(",\n");
+            }
+            sb.append(columnString);
+        }
+        return sb.toString();
+    }
+
+    private static String getColumnInfoString(ColumnDef cDef) {
+        String colName = cDef.getColumnDefName().toString();
+        boolean isArrayType = cDef.getDataType().isArrayType();
+        String type = cDef.getDataType().getSqlTypeName();
+        Integer maxLength = cDef.getMaxLength();
+        Integer arrSize = cDef.getArraySize();
+        Integer scale = cDef.getScale();
+        StringBuilder buf = new StringBuilder(colName);
+        buf.append(' ');
+        if (isArrayType) {
+            String arrayPrefix = type.split("\\s+")[0];
+            buf.append(arrayPrefix);
+            appendMaxLengthAndScale(buf, maxLength, scale);
+            buf.append(' ');
+            buf.append("ARRAY");
+            if (arrSize != null) {
+                buf.append('[');
+                buf.append(arrSize);
+                buf.append(']');
+            }
+        } else {
+            buf.append(type);
+            appendMaxLengthAndScale(buf, maxLength, scale);
+        }
+
+        if (!cDef.isNull()) {
+            buf.append(' ');
+            buf.append("NOT NULL");
+        }
+
+        return buf.toString();
+    }
+
+    private static void appendMaxLengthAndScale(StringBuilder buf, Integer maxLength, Integer scale){
+        if (maxLength != null) {
+            buf.append('(');
+            buf.append(maxLength);
+            if (scale != null) {
+                buf.append(',');
+                buf.append(scale); // has both max length and scale. For ex- decimal(10,2)
+            }
+            buf.append(')');
+        }
+    }
+
+    private static void appendProperties(StringBuffer sb,
+            ListMultimap<String, Pair<String, Object>> props) {
+        if (props != null && !props.isEmpty()) {
+            sb.append("\n");
+            for (Map.Entry<String, Pair<String, Object>> entry : props.entries()) {
+                sb.append(entry.getValue().getFirst()).append("=")
+                        .append(entry.getValue().getSecond());
+                sb.append(",");
+            }
+            sb.deleteCharAt(sb.length()-1);
+        }
+    }
+}
diff --git a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSynthesisProcessor.java b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSynthesisProcessor.java
new file mode 100644
index 0000000..7c8e849
--- /dev/null
+++ b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaSynthesisProcessor.java
@@ -0,0 +1,235 @@
+/*
+ * 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.phoenix.schema;
+
+import org.apache.phoenix.parse.ColumnDefInPkConstraint;
+import org.apache.phoenix.parse.PrimaryKeyConstraint;
+import org.apache.phoenix.thirdparty.com.google.common.collect.ArrayListMultimap;
+import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.parse.AddColumnStatement;
+import org.apache.phoenix.parse.BindableStatement;
+import org.apache.phoenix.parse.ColumnDef;
+import org.apache.phoenix.parse.ColumnName;
+import org.apache.phoenix.parse.CreateIndexStatement;
+import org.apache.phoenix.parse.CreateTableStatement;
+import org.apache.phoenix.parse.DropColumnStatement;
+import org.apache.phoenix.parse.DropIndexStatement;
+import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.parse.SQLParser;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.phoenix.schema.SchemaSQLUtil.getCreateIndexSQL;
+import static org.apache.phoenix.schema.SchemaSQLUtil.getCreateTableSQL;
+
+public class SchemaSynthesisProcessor implements SchemaProcessor {
+    public static final String
+            ENTITY_NAME_IN_BASE_AND_ALTER_DDL_DON_T_MATCH =
+            "Entity name in base and alter DDL don't match";
+    public static final String
+            UNSUPPORTED_DDL_EXCEPTION =
+            "SchemaTool in Synth mode is supported for CREATE TABLE/VIEW/INDEX ddls";
+    private final String ddlFile;
+
+    public SchemaSynthesisProcessor(String ddlFile) {
+        this.ddlFile = ddlFile;
+    }
+
+    @Override
+    public String process() throws Exception {
+        List <String> allDDL = getQueriesFromFile(ddlFile);
+        String ddl = null;
+        for(String s : allDDL) {
+            ddl = synthesize(ddl, s);
+        }
+        return ddl == null ? "" :ddl;
+    }
+
+    private String synthesize(String baseDDL, String nextDDL) throws Exception {
+        if (baseDDL == null && nextDDL != null) {
+            BindableStatement bStmt = new SQLParser(nextDDL).parseStatement();
+            if (bStmt instanceof CreateTableStatement || bStmt instanceof CreateIndexStatement) {
+                return nextDDL;
+            }
+            throw new Exception(UNSUPPORTED_DDL_EXCEPTION);
+        }
+        BindableStatement createStatement = new SQLParser(baseDDL).parseStatement();
+        BindableStatement alterStatement = new SQLParser(nextDDL).parseStatement();
+        if (createStatement instanceof CreateTableStatement) {
+            CreateTableStatement newCreateStmt = null;
+            CreateTableStatement createStmt = (CreateTableStatement) createStatement;
+            if (alterStatement instanceof AddColumnStatement) {
+                newCreateStmt =
+                        getCreateTableStatement((AddColumnStatement) alterStatement, createStmt);
+            } else if (alterStatement instanceof DropColumnStatement) {
+                newCreateStmt =
+                        getCreateTableStatement((DropColumnStatement) alterStatement, createStmt);
+            } else if (alterStatement instanceof DropTableStatement) {
+                return null;
+            }
+            return getCreateTableSQL(newCreateStmt);
+        } else if (createStatement instanceof CreateIndexStatement) {
+            if (alterStatement instanceof DropIndexStatement) {
+                return null;
+            }
+            CreateIndexStatement newCreateIndexStmt =
+                    getCreateIndexStatement(alterStatement, (CreateIndexStatement) createStatement);
+            return getCreateIndexSQL(newCreateIndexStmt);
+        } else {
+            throw new Exception(UNSUPPORTED_DDL_EXCEPTION);
+        }
+    }
+
+    private CreateIndexStatement getCreateIndexStatement(BindableStatement alterStatement, CreateIndexStatement createStatement) throws Exception {
+        CreateIndexStatement newCreateIndexStmt = null;
+        String tableName = createStatement.getIndexTableName().toString();
+        String tableNameInAlter = ((AddColumnStatement)alterStatement).getTable().toString().trim();
+        sanityCheck(tableName, tableNameInAlter);
+        AddColumnStatement addStmt = (AddColumnStatement) alterStatement;
+        if (addStmt.getColumnDefs() == null) {
+            ListMultimap<String, Pair<String, Object>>
+                    finalProps =
+                    getEffectiveProperties(addStmt, createStatement.getProps());
+            newCreateIndexStmt = new CreateIndexStatement(createStatement, finalProps);
+        }
+        return newCreateIndexStmt;
+    }
+
+    private CreateTableStatement getCreateTableStatement(DropColumnStatement alterStatement,
+            CreateTableStatement createStmt) throws Exception {
+        CreateTableStatement newCreateStmt = null;
+        String tableName = createStmt.getTableName().toString();
+        String tableNameInAlter = alterStatement.getTable().toString().trim();
+        sanityCheck(tableName, tableNameInAlter);
+        List<ColumnDef> oldColumnDef = createStmt.getColumnDefs();
+        List<ColumnDef> newColumnDef = new ArrayList<>();
+        newColumnDef.addAll(oldColumnDef);
+        DropColumnStatement dropStmt = alterStatement;
+        for(ColumnName cName : dropStmt.getColumnRefs()) {
+            for(ColumnDef colDef : oldColumnDef) {
+                if (colDef.getColumnDefName().equals(cName)) {
+                    newColumnDef.remove(colDef);
+                    break;
+                }
+            }
+        }
+        newCreateStmt = new CreateTableStatement(createStmt, newColumnDef);
+        return newCreateStmt;
+    }
+
+    private CreateTableStatement getCreateTableStatement(AddColumnStatement alterStatement,
+            CreateTableStatement createStmt) throws Exception {
+        CreateTableStatement newCreateStmt = null;
+        String tableName = createStmt.getTableName().toString();
+        String tableNameInAlter = alterStatement.getTable().toString().trim();
+        sanityCheck(tableName, tableNameInAlter);
+        AddColumnStatement addStmt = alterStatement;
+        List<ColumnDef> oldColDef = createStmt.getColumnDefs();
+        List<ColumnDef> newColDef = new ArrayList<>();
+        if (addStmt.getColumnDefs() == null) {
+            ListMultimap<String, Pair<String, Object>>
+                    finalProps = getEffectiveProperties(addStmt, createStmt.getProps());
+            newCreateStmt = new CreateTableStatement(createStmt, finalProps, oldColDef);
+        } else {
+            newColDef.addAll(oldColDef);
+            newColDef.addAll(addStmt.getColumnDefs());
+            PrimaryKeyConstraint oldPKConstraint = createStmt.getPrimaryKeyConstraint();
+            List<ColumnDefInPkConstraint> pkList = new ArrayList<>();
+            for(Pair<ColumnName, SortOrder> entry : oldPKConstraint.getColumnNames()) {
+                ColumnDefInPkConstraint cd = new
+                        ColumnDefInPkConstraint(entry.getFirst(), entry.getSecond(), oldPKConstraint.isColumnRowTimestamp(entry
+                .getFirst()));
+                pkList.add(cd);
+            }
+            for(ColumnDef cd : addStmt.getColumnDefs()) {
+                if(cd.isPK()) {
+                    ColumnDefInPkConstraint cdpk = new
+                            ColumnDefInPkConstraint(cd.getColumnDefName(), cd.getSortOrder(), cd.isRowTimestamp());
+                    pkList.add(cdpk);
+                }
+            }
+            PrimaryKeyConstraint pkConstraint = new
+                    PrimaryKeyConstraint(oldPKConstraint.getName(), pkList);
+            newCreateStmt = new CreateTableStatement(createStmt, pkConstraint, newColDef);
+        }
+        return newCreateStmt;
+    }
+
+    private void sanityCheck(String tableName, String tableNameInAlter) throws Exception {
+        if (!tableName.equalsIgnoreCase(tableNameInAlter)) {
+            throw new Exception(ENTITY_NAME_IN_BASE_AND_ALTER_DDL_DON_T_MATCH);
+        }
+    }
+
+    private ListMultimap<String, Pair<String, Object>> getEffectiveProperties(
+            AddColumnStatement addStmt, ListMultimap<String, Pair<String, Object>> oldProps) {
+        Map<String, Object> oldPropMap = new HashMap();
+        Map<String, Object> changePropMap = new HashMap();
+
+        for (Pair<String, Object> value : oldProps.values()) {
+            oldPropMap.put(value.getFirst(),value.getSecond());
+        }
+        for (Pair<String, Object> value : addStmt.getProps().values()) {
+            changePropMap.put(value.getFirst(),value.getSecond());
+        }
+
+        oldPropMap.putAll(changePropMap);
+        ListMultimap<String, Pair<String, Object>>
+                finalProps =
+                ArrayListMultimap.<String, Pair<String, Object>>create();
+        for (Map.Entry<String, Object> entry : oldPropMap.entrySet()) {
+            finalProps.put("", Pair.newPair(entry.getKey(), entry.getValue()));
+        }
+        return finalProps;
+    }
+
+    private List<String> getQueriesFromFile(String ddlFile) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        File file = new File(ddlFile);
+        BufferedReader br = new BufferedReader(new FileReader(file));
+        String st;
+        while ((st = br.readLine()) != null) {
+            sb.append(st).append("\n");
+        }
+        String trimmedQuery = sb.toString().trim();
+        if (trimmedQuery.contains("/*") && trimmedQuery.contains("*/")) {
+            trimmedQuery = trimmedQuery.substring(trimmedQuery.lastIndexOf("*/") + 2);
+        }
+        String [] queries = trimmedQuery.split(";");
+        List<String> output = new ArrayList<>();
+        for(String query: queries) {
+            StringBuilder newSb = new StringBuilder(query);
+            char lastChar = newSb.charAt(newSb.length() - 1);
+            // DDL in the file should not have a ; at the end
+            // remove the last char if it is ; or \n
+            if (lastChar == '\n' || lastChar == ';') {
+                newSb.deleteCharAt(newSb.length() - 1);
+            }
+            output.add(newSb.toString().trim());
+        }
+        return output;
+    }
+}
diff --git a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionTool.java b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaTool.java
similarity index 66%
rename from phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionTool.java
rename to phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaTool.java
index 15dca25..ad40cd5 100644
--- a/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaExtractionTool.java
+++ b/phoenix-tools/src/main/java/org/apache/phoenix/schema/SchemaTool.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.schema;
 
+import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec;
 import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLine;
 import org.apache.phoenix.thirdparty.org.apache.commons.cli.CommandLineParser;
 import org.apache.phoenix.thirdparty.org.apache.commons.cli.DefaultParser;
@@ -35,13 +36,18 @@ import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class SchemaExtractionTool extends Configured implements Tool {
+public class SchemaTool extends Configured implements Tool {
 
-    private static final Logger LOGGER = LoggerFactory.getLogger(SchemaExtractionTool.class);
+    private static final Logger LOGGER = LoggerFactory.getLogger(SchemaTool.class);
     private static final Option HELP_OPTION = new Option("h", "help",
             false, "Help");
+    private static final Option MODE_OPTION = new Option("m", "mode", true,
+            "[Required] Takes either synth or extract value");
+    private static final Option DDL_OPTION = new Option("d", "ddl", true,
+            "[Required with synth mode] SQL file that has one or more ddl statements"
+                    + " for the same entity");
     private static final Option TABLE_OPTION = new Option("tb", "table", true,
-            "[Required] Table name ex. table1");
+            "[Required with extract mode] Table name ex. table1");
     private static final Option SCHEMA_OPTION = new Option("s", "schema", true,
             "[Optional] Schema name ex. schema");
     private static final Option TENANT_OPTION = new Option("t", "tenant", true,
@@ -50,18 +56,27 @@ public class SchemaExtractionTool extends Configured implements Tool {
     private String pTableName;
     private String pSchemaName;
     private String tenantId;
+    private Enum mode;
 
-    public static Configuration conf;
+    protected static Configuration conf;
     private String output;
+    private String ddlFile;
+    private String alterDDLFile;
 
     @Override
     public int run(String[] args) throws Exception {
         populateToolAttributes(args);
-        conf = HBaseConfiguration.addHbaseResources(getConf());
-        SchemaExtractionProcessor processor = new SchemaExtractionProcessor(tenantId,
-                conf, pSchemaName, pTableName);
+        SchemaProcessor processor=null;
+        if(Mode.SYNTH.equals(mode)) {
+            processor = new SchemaSynthesisProcessor(ddlFile);
+        } else if(Mode.EXTRACT.equals(mode)) {
+            conf = HBaseConfiguration.addHbaseResources(getConf());
+            processor = new SchemaExtractionProcessor(tenantId, conf, pSchemaName, pTableName);
+        } else {
+            throw new Exception(mode+" is not accepted, provide [synth or extract]");
+        }
         output = processor.process();
-        LOGGER.info("Extracted DDL: " + output);
+        LOGGER.info("Effective DDL with " + mode.toString() +": " + output);
         return 0;
     }
 
@@ -72,10 +87,12 @@ public class SchemaExtractionTool extends Configured implements Tool {
     private void populateToolAttributes(String[] args) {
         try {
             CommandLine cmdLine = parseOptions(args);
+            mode = Mode.valueOf(cmdLine.getOptionValue(MODE_OPTION.getOpt()));
+            ddlFile = cmdLine.getOptionValue(DDL_OPTION.getOpt());
             pTableName = cmdLine.getOptionValue(TABLE_OPTION.getOpt());
             pSchemaName = cmdLine.getOptionValue(SCHEMA_OPTION.getOpt());
             tenantId = cmdLine.getOptionValue(TENANT_OPTION.getOpt());
-            LOGGER.info("Schema Extraction Tool initiated: " + StringUtils.join( args, ","));
+            LOGGER.info("Schema Tool initiated: " + StringUtils.join( args, ","));
         } catch (IllegalStateException e) {
             printHelpAndExit(e.getMessage(), getOptions());
         }
@@ -91,19 +108,35 @@ public class SchemaExtractionTool extends Configured implements Tool {
             printHelpAndExit("severe parsing command line options: " + e.getMessage(),
                     options);
         }
+        if(cmdLine == null) {
+            printHelpAndExit("parsed command line object is null", options);
+        }
         if (cmdLine.hasOption(HELP_OPTION.getOpt())) {
             printHelpAndExit(options, 0);
         }
-        if (!(cmdLine.hasOption(TABLE_OPTION.getOpt()))) {
-            throw new IllegalStateException("Table name should be passed "
+        if (!(cmdLine.hasOption(TABLE_OPTION.getOpt()))
+                && cmdLine.getOptionValue(MODE_OPTION.getOpt()).equalsIgnoreCase(Mode.EXTRACT.toString())) {
+            throw new IllegalStateException("Table name should be passed with EXTRACT mode"
                     +TABLE_OPTION.getLongOpt());
         }
+        if ((!(cmdLine.hasOption(DDL_OPTION.getOpt())))
+                && cmdLine.getOptionValue(MODE_OPTION.getOpt()).equalsIgnoreCase(Mode.SYNTH.toString())) {
+            throw new IllegalStateException("ddl option should be passed with SYNTH mode"
+                    + DDL_OPTION.getLongOpt());
+        }
         return cmdLine;
     }
 
+    enum Mode {
+        SYNTH,
+        EXTRACT
+    }
+
     private Options getOptions() {
         final Options options = new Options();
         options.addOption(TABLE_OPTION);
+        options.addOption(MODE_OPTION);
+        options.addOption(DDL_OPTION);
         SCHEMA_OPTION.setOptionalArg(true);
         options.addOption(SCHEMA_OPTION);
         TENANT_OPTION.setOptionalArg(true);
@@ -123,7 +156,7 @@ public class SchemaExtractionTool extends Configured implements Tool {
     }
 
     public static void main (String[] args) throws Exception {
-        int result = ToolRunner.run(new SchemaExtractionTool(), args);
+        int result = ToolRunner.run(new SchemaTool(), args);
         System.exit(result);
     }
 }