Forráskód Böngészése

AMBARI-15915. SQL constraints: Inline constraints and name them in CREATE table. (Balazs Bence Sari via stoader)

Balazs Bence Sari 9 éve
szülő
commit
346dfe7eb7

A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 303 - 318
ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 302 - 322
ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 295 - 314
ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 297 - 324
ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 344 - 309
ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 300 - 319
ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql


A különbségek nem kerülnek megjelenítésre, a fájl túl nagy
+ 294 - 403
ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql


+ 525 - 0
ambari-server/src/test/java/org/apache/ambari/server/orm/db/DDLTestUtils.java

@@ -0,0 +1,525 @@
+/*
+ * 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.ambari.server.orm.db;
+
+import com.google.common.base.*;
+import com.google.common.base.Optional;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.collect.*;
+import com.google.common.io.Resources;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.util.*;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * <p>
+ * Utility to parse DDL scripts for verification and comparision. Parsing is not complete, only things relevant for
+ * the unit tests are considered, e.g. the column data types are not captured as they may differ across different SQL
+ * dialects.
+ * </p>
+ * <p>
+ * Parsing is mostly done by regexp matches, so the parser has limited capabilities. The following known restrictions apply:
+ * </p>
+ * <ul>
+ *   <li>Multiple statements in a single line are not supported, e.g: CREATE TABLE X(...); CREATE TABLE Y(...)</li>
+ *   <li>Mutiple definitions in the same line within a create statement is supported though, e.g: name varchar(255), id bigint, ...</li>
+ *   <li>Statements must be terminated by ;</li>
+ * </ul>
+ *
+ */
+public class DDLTestUtils {
+  private static final Logger LOG = LoggerFactory.getLogger(DDLTestUtils.class);
+
+  // These patterns are used during the initial line by line parsing of a DDL.
+  // The patterns help
+  // 1. filter out irrelevant lines (comment, empty line, go/commit),
+  // 2. recognize statement starters (create table, alter table, create index)
+  // 3. recognize statement terminators (;)
+  private static final Pattern CommentLine =  Pattern.compile("^\\s*--.*");
+  private static final Pattern EmptyLine =    Pattern.compile("^\\s*$");
+  private static final Pattern CommitLine =   Pattern.compile("^\\s*(go|commit).*$");
+  private static final Pattern CreateTable =  Pattern.compile("^\\s*create\\s+table.*$");
+  private static final Pattern AlterTable =   Pattern.compile("^\\s*alter\\s+table.*$");
+  private static final Pattern CreateIndex =  Pattern.compile("^\\s*create\\s+index.*$");
+  private static final Pattern EndStatement = Pattern.compile("(.*\\;)\\s*$");
+
+  // These patterns are used to match column/constraint definitons in a create table statement
+  // to capture the table name
+  private static final Pattern TableName =   Pattern.compile("^\\s*create table\\s+([\\w\\.\\_]+).*$");
+  // to capture the name and columns in a primary key
+  private static final Pattern PK =          Pattern.compile("^.*constraint\\s+([\\w\\.\\_]+)\\s+primary\\s+key\\s*\\(([^\\)]+)\\).*$");
+  // to capture the name and columns in a clusterd primary key
+  private static final Pattern PKClustered = Pattern.compile("^.*constraint\\s+([\\w\\.\\_]+)\\s+primary\\s+key\\s+clustered\\s*\\(([^\\)]+)\\).*$");
+  // to capture the name and columns in a unique constraint
+  private static final Pattern UQ =          Pattern.compile("^.*constraint\\s+([\\w\\.\\_]+)\\s+unique\\s*\\(([^\\)]+)\\).*$");
+  // to capture the name and columns and the referred columnd in a foreign key
+  private static final Pattern FK =          Pattern.compile("^.*constraint\\s+([\\w\\.\\_]+)\\s+foreign\\s+key\\s*\\(([^\\)]*)\\)\\s*references\\s+([\\w\\_\\.]+)\\s*\\(([^\\)]+)\\).*$");
+  // to capture the name of a columns
+  private static final Pattern Col =         Pattern.compile("^\\s*([\\`\\\"\\[\\]\\w\\.\\_]+)\\s+.*$");
+  // to capture column lists within the create table statement, such as the column list in "primary key (name, id)"
+  // in such lists commas are replaced with a | so that we can reliably use comma as a definition separator within the create table statement
+  private static final Pattern InnerList =   Pattern.compile("(\\([^\\(^\\)]+\\))");
+
+  // These patterns represent Unnamed constraints
+  private static final Pattern UnnamedPK =          Pattern.compile("^\\s*primary\\s+key[\\sclustered]*\\(([^\\)]+)\\).*$"); // e.g: primary key [clustered] (name)
+  private static final Pattern UnnamedUQ =          Pattern.compile("^\\s*unique\\s*\\(([^\\)]+)\\).*$"); // e.g: unique (name)
+  private static final Pattern UnnamedFK =          Pattern.compile("^\\s*foreign\\s+key\\s*\\(([^\\)]+)\\)\\s*references\\s+([\\w\\_\\.]+)\\s*\\(([^\\)]+)\\).*$"); // e.g: foreign key (name) references other_table(name)
+  private static final Pattern PKColumn =          Pattern.compile("^.*[\\w\\.\\_]+\\s.*primary\\s+key[\\sclustered\\,\\;\\)]*$"); // e.g: name varchar(255) not null primary key [clustered]
+  private static final Pattern UQColumn =          Pattern.compile("^\\s*[\\w\\.\\_]+\\s.*unique[\\s\\;\\,\\)]*$"); // e.g: name varchar(255) not null unique
+  private static final List<Pattern> CheckedUnnamedConstraints = ImmutableList.of(UnnamedPK);
+  private static final List<Pattern> UncheckedUnnamedConstraints = ImmutableList.of(UnnamedUQ, UnnamedFK, PKColumn, UQColumn);
+
+  private static final LoadingCache<String, DDL> ddlCache = CacheBuilder.newBuilder().build(
+    new CacheLoader<String, DDL>() {
+      @Override public DDL load(String key) throws Exception { return loadDdl(key); }
+    }
+  );
+
+  /**
+   * List of supported databases.
+   */
+  public static final List<String> DATABASES = ImmutableList.of(
+    "Derby",
+    "MySQL",
+    "Oracle",
+    "Postgres",
+    "Postgres-EMBEDDED",
+    "SQLAnywhere",
+    "SQLServer");
+
+  /**
+   * Loads and parses the DDL for a specific database type (e.g. Postgres). Already loaded DDL's are cached.
+   * @param dbType One of the supported databases @see #DATABASES
+   * @return the parsed DDL
+   * @throws Exception if exception occurs during loading/parsing
+   */
+  public static DDL getDdl(String dbType) throws Exception{
+    return ddlCache.get(dbType);
+  }
+
+  private static URL getDdlUrl(String dbType) {
+    return Resources.getResource("Ambari-DDL-" + dbType + "-CREATE.sql");
+  }
+
+  private static List<String> loadFile(String dbType) throws Exception {
+    List<String> lines = Resources.readLines(getDdlUrl(dbType), Charsets.UTF_8);
+    List<String> replaced = new ArrayList<>(lines.size());
+    for (String line: lines) { replaced.add(line.toLowerCase()); }
+    return replaced;
+  }
+
+  /**
+   * Groups the load DDL file into statements. Currently CREATE TABLE's, ALTER TABLE's and CREATE INDEX-es
+   * are considered statements. Multiple statements in a single line are not supported. Comments, empty lines and
+   * GO / COMMIT commands are discarded.
+   *
+   * @param ddlFile the loaded DDL file (as list of strings)
+   * @return a list of strings containing the statements
+   */
+  private static List<String> groupStatements(List<String> ddlFile) {
+    List<String> statements = new ArrayList<>();
+    Optional<ArrayList<String>> currentStmt = Optional.absent();
+    for (String line: ddlFile) {
+      // These lines should be skipped
+      if (CommentLine.matcher(line).matches() ||
+          EmptyLine.matcher(line).matches() ||
+          CommitLine.matcher(line).matches());
+      // These lines indicate the start of a CREATE TABLE / ALTER TABLE / CREATE INDEX statement
+      else if (CreateTable.matcher(line).matches() ||
+          AlterTable.matcher(line).matches() ||
+          CreateIndex.matcher(line).matches()) {
+        // Prepare to collect subsequent lines as part of the new statement
+        if(currentStmt.isPresent()) throw new IllegalStateException(
+            "Unfinished statement: " + currentStmt.get() + "\nnew statement: " +line);
+        currentStmt = Optional.of(new ArrayList<String>());
+        currentStmt.get().add(stripComment(line));
+        // If the statement is a one liner, close it right away
+        if (line.contains(";")) {
+          statements.add(Joiner.on(' ').join(currentStmt.get()));
+          currentStmt = Optional.absent();
+        }
+      }
+      // Process terminating line (containing ;): add to the current statement and close current statement
+      else if (currentStmt.isPresent() && EndStatement.matcher(line).matches()) {
+        currentStmt.get().add(stripComment(line));
+        statements.add(Joiner.on(' ').join(currentStmt.get()));
+        currentStmt = Optional.absent();
+      }
+      // Collect all other lines as part of the current statement
+      else if (currentStmt.isPresent()) {
+        currentStmt.get().add(stripComment(line));
+      }
+    }
+    return statements;
+  }
+
+  private static String stripComment(String line) {
+    return line.contains("--") ? line.substring(0, line.indexOf("--")) : line;
+  }
+
+  private static Collection<String> toColumns(String cols) {
+    List<String> columns = new ArrayList<>();
+    for (String col: Splitter.on('|').split(cols)) {
+      columns.add( stripPrefixQuotationAndBrackets(col.trim()));
+    }
+    return columns;
+  }
+
+  /**
+   * Strips out quotation characters ('"[]) and schema prefixes from identifiers
+   * (table / columns / constraint names)
+   * @param input an identifier
+   * @return the important part of the identifier
+   */
+  private static String stripPrefixQuotationAndBrackets(String input) {
+    String output = input.replaceAll("[\\`\\\"\\[\\]]", "").replaceAll("[^\\.]*\\.", "");
+    return output;
+  }
+
+  private static Optional<String> firstMatchingGroup(Pattern p, String s) {
+    Matcher m = p.matcher(s);
+    if (m.matches()) {
+      return Optional.of(m.group(1));
+    }
+    else {
+      return Optional.absent();
+    }
+  }
+
+  private static Map<String, Table> parseTableDefs(List<String> statements) {
+    // Find all CREATE TABLE statements
+    List<String> createTables = new ArrayList<>();
+    for (String stmt: statements) {
+      if (stmt.matches(".*create\\s+table.*")) {
+        String content = stmt.substring(stmt.indexOf('(') + 1, stmt.lastIndexOf(')'));
+        // Replace , with | within PK/FK/UQ definitions so that we will be able to partition column/constraint definitions by ,
+        Matcher m = InnerList.matcher(content);
+        while (m.find()) {
+          String innerList = m.group();
+          stmt = stmt.replace(innerList, innerList.replaceAll("\\,", "|"));
+        }
+        createTables.add(stmt);
+      }
+    }
+    List<Table> tables = new ArrayList<>();
+    // Parse CREATE TABLE statements
+    for(String ct: createTables) {
+      String tableName = stripPrefixQuotationAndBrackets(firstMatchingGroup(TableName, ct).get());
+      List<String> columns = new ArrayList<>();
+      Optional<SimpleConstraint> pk = Optional.absent();
+      List<FKConstraint> fks = new ArrayList<>();
+      List<SimpleConstraint> uqs = new ArrayList<>();
+      final String innerPart = ct.substring(ct.indexOf('(') + 1, ct.lastIndexOf(')'));
+      for (String definition: Splitter.on(',').split(innerPart)) {
+        definition = definition.trim();
+        assertNounnamedConstraint(tableName, definition);
+        Matcher pkMatcher = PK.matcher(definition);
+        Matcher pkClustMatcher = PKClustered.matcher(definition);
+        Matcher unnamedPkMatcher = UnnamedPK.matcher(definition);
+        Matcher pkColumnMatcher = PKColumn.matcher(definition);
+        Matcher fkMatcher = FK.matcher(definition);
+        Matcher uqMatcher = UQ.matcher(definition);
+        Matcher unnamedFkMatcher = UnnamedFK.matcher(definition);
+        Matcher unnamedUqMatcher = UnnamedUQ.matcher(definition);
+        Matcher uqColumnMatcher = UQColumn.matcher(definition);
+        Matcher colMatcher = Col.matcher(definition);
+        if (pkMatcher.matches()) {
+          pk = Optional.of(Constraint.pk(pkMatcher.group(1),toColumns(pkMatcher.group(2))));
+        } else if (pkMatcher.matches()) {
+          pk = Optional.of(Constraint.pk(stripPrefixQuotationAndBrackets(pkMatcher.group(1)),toColumns(pkMatcher.group(2))));
+        } else if (pkClustMatcher.matches()) {
+          pk = Optional.of(Constraint.pk(stripPrefixQuotationAndBrackets(pkClustMatcher.group(1)),toColumns(pkClustMatcher.group(2))));
+        } else if (unnamedPkMatcher.matches()) {
+          pk = Optional.of(Constraint.pk("<default>",toColumns(unnamedPkMatcher.group(1))));
+        } else if (fkMatcher.matches()) {
+          fks.add(Constraint.fk(fkMatcher.group(1), toColumns(fkMatcher.group(2)), stripPrefixQuotationAndBrackets(fkMatcher.group(3)), toColumns(fkMatcher.group(4))));
+        } else if (unnamedFkMatcher.matches()) {
+          fks.add(Constraint.fk("<default>", toColumns(unnamedFkMatcher.group(1)), stripPrefixQuotationAndBrackets(unnamedFkMatcher.group(2)), toColumns(unnamedFkMatcher.group(3))));
+        } else if (uqMatcher.matches()) {
+          uqs.add(Constraint.uq(stripPrefixQuotationAndBrackets(uqMatcher.group(1)),toColumns(uqMatcher.group(2))));
+        } else if (unnamedUqMatcher.matches()) {
+          uqs.add(Constraint.uq("<default>", toColumns(unnamedUqMatcher.group(1))));
+        } else if (colMatcher.matches()) {
+          String colName = stripPrefixQuotationAndBrackets(colMatcher.group(1));
+          columns.add(colName);
+          // column definitions can include PK/UQ declaration, e.g: x integer not null primary key
+          if (pkColumnMatcher.matches()) {
+            pk = Optional.of(Constraint.pk("<default>", Collections.singleton(colName)));
+          } else if (uqColumnMatcher.matches()) {
+            uqs.add(Constraint.uq("<default>", Collections.singleton(colName)));
+          }
+        } else {
+          LOG.warn("Unexpected definition: {}, context: {}", definition, ct);
+        }
+      }
+      if (columns.isEmpty()) {
+        throw new IllegalStateException("No columns found in table " + tableName);
+      }
+      checkDupes("columns of table " + tableName, columns);
+      tables.add(new Table(tableName,
+          ImmutableSet.copyOf(columns),
+          pk,
+          ImmutableSet.copyOf(fks),
+          ImmutableSet.copyOf(uqs)));
+    }
+    Map<String, Table> tableMap = Maps.newHashMap();
+    for(Table t: tables) {
+      if (tableMap.containsKey(t.name)) throw new IllegalStateException("Duplicate table definition: " + t.name);
+      tableMap.put(t.name, t);
+    }
+    return tableMap;
+  }
+
+  private static void checkDupes(String objectName, List<? extends Object> items) {
+    Set<Object> set = Sets.newHashSet(items);
+    if (set.size() < items.size()) {
+      throw new IllegalStateException(String.format("Duplicates found in %s: %s", objectName, Iterables.toString(items)));
+    }
+  }
+
+  /**
+   * Currently we only fail on unnamed primary keys.
+   * @param tableName
+   * @param definition
+   */
+  private static void assertNounnamedConstraint(String tableName, String definition) {
+    if (tableName.contains("qrtz")) {
+      LOG.debug("Skipp checking quartz table: {}", tableName);
+    }
+    else {
+      for (Pattern unnamedConstraint: CheckedUnnamedConstraints) {
+        if (unnamedConstraint.matcher(definition).matches()) {
+          throw new IllegalStateException(
+              String.format("Found invalid (unnamed) constraint in table %s: %s", tableName, definition));
+        }
+      }
+      for (Pattern unnamedConstraint: UncheckedUnnamedConstraints) {
+        if (unnamedConstraint.matcher(definition).matches()) {
+          LOG.info("Found unnamed constraint in table {}: {}", tableName, definition);
+        }
+      }
+    }
+  }
+
+  private static DDL loadDdl(String dbType) throws Exception {
+    List<String> lines = loadFile(dbType);
+    List<String> statements = groupStatements(lines);
+    Map<String, Table> tables = parseTableDefs(statements);
+    List<String> alterTables = new ArrayList<>();
+    for (String stmt: statements) {
+      if (stmt.matches(".*alter\\s+table.*")) alterTables.add(stmt);
+    }
+    return new DDL(dbType, tables, alterTables);
+  }
+
+}
+
+/**
+ * Represents a DDL
+ */
+class DDL {
+  final String dbType;
+  final Map<String, Table> tables;
+  final List<String> alterTables;
+
+  Set<String> tableNames() { return tables.keySet(); }
+
+  DDL(String dbType, Map<String, Table> tables, List<String> alterTables) {
+    this.dbType = dbType;
+    this.tables = tables;
+    this.alterTables = alterTables;
+  }
+}
+
+/**
+ * Represents a datbase table
+ */
+class Table {
+  final String name;
+  final ImmutableSet<String> columns;
+  final Optional<SimpleConstraint> primaryKey;
+  final ImmutableSet<FKConstraint> foreignKeys;
+  final ImmutableSet<SimpleConstraint> uniqueConstraints;
+
+  Table(String name, Set<String> columns, Optional<SimpleConstraint> primaryKey, Set<FKConstraint> foreignKeys, Set<SimpleConstraint> uniqueConstraints) {
+    this.name = name;
+    this.columns =
+        (columns instanceof ImmutableSet) ? (ImmutableSet<String>)columns : ImmutableSet.copyOf(columns);
+    this.primaryKey = primaryKey;
+    this.foreignKeys =
+        (foreignKeys instanceof ImmutableSet) ? (ImmutableSet<FKConstraint>)foreignKeys : ImmutableSet.copyOf(foreignKeys);
+    this.uniqueConstraints =
+        (uniqueConstraints instanceof ImmutableSet) ? (ImmutableSet<SimpleConstraint>) uniqueConstraints : ImmutableSet.copyOf(uniqueConstraints);
+  }
+
+  @Override
+  public String toString() {
+    return String.format("TABLE name: %s, columns: %s, pk: %s, fks: %s, uqs: %s",
+        name, Iterables.toString(columns), primaryKey, Iterables.toString(foreignKeys), Iterables.toString(uniqueConstraints));
+  }
+}
+
+/**
+ * Represents a constraint.
+ */
+abstract class Constraint<ContentType> {
+  abstract String name();
+  abstract ContentType content();
+
+  static SimpleConstraint pk(String name, Collection<String> columns) {
+    Preconditions.checkArgument(!columns.isEmpty(), "Columns must not be empty.");
+    return new SimpleConstraint(name, "PK", columns);
+  }
+
+  static SimpleConstraint uq(String name, Collection<String> columns) {
+    Preconditions.checkArgument(!columns.isEmpty(), "Columns must not be empty.");
+    return new SimpleConstraint(name, "PK", columns);
+  }
+
+  static FKConstraint fk(String name, Collection<String> columns, String referredTableName, Collection<String> referredColumns) {
+    Preconditions.checkArgument(!columns.isEmpty(), "Columns must not be empty.");
+    Preconditions.checkArgument(!referredColumns.isEmpty(), "Referred columns must not be empty.");
+    return new FKConstraint(name, columns, referredTableName, referredColumns);
+  }
+
+}
+
+/**
+ * Represents a simple constraint (PK/UQ)
+ */
+class SimpleConstraint extends Constraint<Set<String>> {
+  final String name;
+  final String type;
+  final ImmutableSet<String> columns; // These have favorable equals/hashcode semantics
+
+  SimpleConstraint(String name, String type, Collection<String> columns) {
+    this.name = name;
+    this.type = type;
+    this.columns = (columns instanceof ImmutableSet) ? (ImmutableSet<String>) columns : ImmutableSet.copyOf(columns);
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public ImmutableSet<String> content() { return columns; }
+
+  @Override public String toString() {
+    return String.format("%s %s [%s]", type, name, Joiner.on(',').join(columns));
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    SimpleConstraint that = (SimpleConstraint) o;
+    if (!name.equals(that.name)) return false;
+    if (!type.equals(that.type)) return false;
+    return columns.equals(that.columns);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name.hashCode();
+    result = 31 * result + type.hashCode();
+    result = 31 * result + columns.hashCode();
+    return result;
+  }
+
+}
+
+class FKConstraintContent {
+  final ImmutableSet<String> columns; // These have favorable equals/hashcode semantics
+  final String referredTable;
+  final ImmutableSet<String> referredColumns; // These have favorable equals/hashcode semantics
+
+  public FKConstraintContent(Collection<String> columns, String referredTable, Collection<String> referredColumns) {
+    this.columns = columns instanceof ImmutableSet ? (ImmutableSet<String>)columns : ImmutableSet.copyOf(columns);
+    this.referredTable = referredTable;
+    this.referredColumns = referredColumns instanceof ImmutableSet ? (ImmutableSet<String>)referredColumns :
+        ImmutableSet.copyOf(referredColumns);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    FKConstraintContent that = (FKConstraintContent) o;
+    if (!columns.equals(that.columns)) return false;
+    if (!referredTable.equals(that.referredTable)) return false;
+    return referredColumns.equals(that.referredColumns);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = columns.hashCode();
+    result = 31 * result + referredTable.hashCode();
+    result = 31 * result + referredColumns.hashCode();
+    return result;
+  }
+
+  @Override public String toString() {
+    return String.format("[%s] --> %s [%s]", Joiner.on(',').join(columns), referredTable, Joiner.on(',').join(referredColumns));
+  }
+
+}
+
+class FKConstraint extends Constraint<FKConstraintContent> {
+  final String name;
+  final FKConstraintContent content;
+
+  FKConstraint(String name, Collection<String> columns, String referredTable, Collection<String> referredColumns) {
+    this.name = name;
+    this.content = new FKConstraintContent(columns, referredTable, referredColumns);
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public FKConstraintContent content() {
+    return content;
+  }
+
+  @Override public String toString() {
+    return String.format("FK name:%s content: %s", name, content);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    FKConstraint that = (FKConstraint) o;
+    if (!name.equals(that.name)) return false;
+    return content.equals(that.content);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = name.hashCode();
+    result = 31 * result + content.hashCode();
+    return result;
+  }
+}

+ 277 - 0
ambari-server/src/test/java/org/apache/ambari/server/orm/db/DDLTests.java

@@ -0,0 +1,277 @@
+/**
+ * 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.ambari.server.orm.db;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+
+/**
+ * Test to check the sanity and conisistence of DDL scripts for different SQL dialects.
+ * (e.g. no unnamed constraints, the same tables with the same columns and constraints must exist)
+ */
+public class DDLTests {
+  private static final Logger LOG = LoggerFactory.getLogger(DDLTestUtils.class);
+
+  private static final int EXPECTED_ALTER_TABLE_COUNT = 1;
+
+  @Test
+  public void testVerifyDerby() throws Exception {
+    verifyDDL("Derby");
+  }
+
+  @Test
+  public void testVerifyPostgres() throws Exception {
+    verifyDDL("Postgres");
+  }
+
+  @Test
+  public void testVerifyPostgresEmbedded() throws Exception {
+    verifyDDL("Postgres-EMBEDDED");
+  }
+
+  @Test
+  public void testVerifyMySQL() throws Exception {
+    verifyDDL("MySQL");
+  }
+
+  @Test
+  public void testVerifyOracle() throws Exception {
+    verifyDDL("Oracle");
+  }
+
+  @Test
+  public void testVerifySqlAnywhere() throws Exception {
+    verifyDDL("SQLAnywhere");
+  }
+
+  @Test
+  public void testVerifyMsSqlServer() throws Exception {
+    verifyDDL("SQLServer");
+  }
+
+  /**
+   * To verify if DDL have certain characteristics:
+   * <ul>
+   *   <li>There are no unnamed constraints</li>
+   *   <li>Alter tables are only used in exceptional cases</li>
+   *   <li>Table and constraint name lenghts doesn't exceed 30 (a restriction imposed by Oracle)</li>
+   * </ul>
+   *
+   */
+  private void verifyDDL(String dbType) throws Exception {
+    LOG.info("Checking DDL for {}", dbType);
+    DDL ddl = DDLTestUtils.getDdl(dbType);
+    printDDLMetrics(ddl);
+
+    // check for unwanted alter tables
+    Assert.assertEquals("Expected count of alter tables mismatch. Please include all constraint definitions in " +
+            "the create table statement, only use alter table in exceptional cases, such as to work around a circular " +
+            "FK dependency. Would another such case occur, please document it in the DDL's and adjust the " +
+            "EXPECTED_ALTER_TABLE_COUNT in this test.",
+        EXPECTED_ALTER_TABLE_COUNT,
+        ddl.alterTables.size());
+
+    // check for too long table/constraint names
+    for (String tableName: ddl.tableNames()) {
+      Assert.assertTrue("Table name exceeds the 30 character limit: " + tableName, tableName.length() <= 30);
+    }
+    for (Table table: ddl.tables.values()) {
+      Assert.assertTrue("PK name exceeds the 30 character limit: " + table.primaryKey,
+          !table.primaryKey.isPresent() || table.primaryKey.get().name().length() <= 30);
+      for (Constraint constr: Sets.union(table.foreignKeys, table.uniqueConstraints)) {
+        Assert.assertTrue("Constraint name exceeds the 30 character limit: " + constr, constr.name().length() <= 30);
+      }
+    }
+    // check for unnamed PK's (skip quartz tables)
+    for (Table table: ddl.tables.values()) {
+      Assert.assertFalse("Unnamed PK exists for table: " + table.name,
+          !table.name.startsWith("qrtz") && table.primaryKey.isPresent() && table.primaryKey.get().name().equals("<default>"));
+      for (Constraint constr: Sets.union(table.foreignKeys, table.uniqueConstraints)) {
+        Assert.assertTrue("Constraint name exceeds the 30 character limit: " + constr, constr.name().length() <= 30);
+      }
+    }
+
+  }
+
+  @Test
+  public void testComparePostgresEmbedded() throws Exception {
+    compareAgainstPostgres("Postgres-EMBEDDED");
+  }
+
+  @Test
+  public void testCompareDerby() throws Exception {
+    compareAgainstPostgres("Derby");
+  }
+
+  @Test
+  public void testCompareOracle() throws Exception {
+    compareAgainstPostgres("Oracle");
+  }
+
+  @Test
+  public void testCompareMySQL() throws Exception {
+    compareAgainstPostgres("MySQL");
+  }
+
+  @Test
+  public void testCompareSQLAnywhere() throws Exception {
+    compareAgainstPostgres("SQLAnywhere");
+  }
+
+  @Test
+  public void testCompareSQLServer() throws Exception {
+    compareAgainstPostgres("SQLServer");
+  }
+
+  static void compareAgainstPostgres(String dbType) throws Exception {
+    LOG.info("Comparing {} against Postgres", dbType);
+    DDL postgres = DDLTestUtils.getDdl("Postgres");
+    DDL other = DDLTestUtils.getDdl(dbType);
+    List<String> diffs = compareDdls(postgres, other);
+    if (diffs.isEmpty()) {
+      LOG.info("Compare OK.");
+    }
+    else {
+      LOG.info("{} differences found:", diffs.size());
+      for (String diff: diffs) { LOG.info(diff); }
+      Assert.fail("Found " + diffs.size() + " differences when comparing " + other + " against Postgres.");
+    }
+  }
+
+  static void printDDLMetrics(DDL ddl) {
+    LOG.info("DDL metrics for {}", ddl.dbType);
+    int colCount = 0;
+    int pkCount = 0;
+    int fkCount = 0;
+    int uqCount = 0;
+    for (Table t: ddl.tables.values()) {
+      colCount += t.columns.size();
+      if (t.primaryKey.isPresent()) pkCount ++;
+      fkCount += t.foreignKeys.size();
+      uqCount += t.uniqueConstraints.size();
+    }
+    LOG.info("Found {} tables", ddl.tables.size());
+    List<String> tableNames = new ArrayList<>();
+    tableNames.addAll(ddl.tableNames());
+    Collections.sort(tableNames);
+    LOG.info("Table names: {}", Joiner.on(',').join(tableNames));
+    LOG.info("Total number of Columns: {}", colCount);
+    LOG.info("Total number of PK's: {}", pkCount);
+    LOG.info("Total number of FK's: {}", fkCount);
+    LOG.info("Total number of UQ's: {}", uqCount);
+    LOG.info("Number of Alter table statements: {}", ddl.alterTables.size());
+
+  }
+
+  static List<String> compareDdls(DDL base, DDL other) {
+    List<String> diffs = new ArrayList<>();
+
+    if (!base.tableNames().equals(other.tableNames())) {
+      Set<String> missingTables = Sets.difference(base.tableNames(), other.tableNames());
+      if (!missingTables.isEmpty()) {
+        diffs.add("Missing tables: " + Joiner.on(", ").join(missingTables));
+      }
+
+      Set<String> extraTables = Sets.difference(other.tableNames(), base.tableNames());
+      if (!extraTables.isEmpty()) {
+        diffs.add("Extra tables: " + Joiner.on(", ").join(extraTables));
+      }
+    }
+
+    Set<String> commonTables = Sets.intersection(base.tableNames(), other.tableNames());
+    for (String tableName: commonTables) {
+      Table baseTable = base.tables.get(tableName);
+      Table otherTable = other.tables.get(tableName);
+      diffs.addAll(
+          compareSets(String.format("Comparing columns of table %s.", tableName), baseTable.columns, otherTable.columns));
+      diffs.addAll(
+          DDLTests.<FKConstraintContent>compareConstraints(tableName, "FK", baseTable.foreignKeys, otherTable.foreignKeys, false));
+      diffs.addAll(
+          DDLTests.<Set<String>>compareConstraints(tableName, "UQ", baseTable.uniqueConstraints, otherTable.uniqueConstraints, false));
+      boolean comparePKName = !tableName.contains("qrtz"); // we are more lenient with quartz tables
+      diffs.addAll(
+          DDLTests.<Set<String>>compareConstraints(tableName, "PK", toSet(baseTable.primaryKey), toSet(otherTable.primaryKey), comparePKName));
+    }
+
+    return diffs;
+  }
+
+  static <T> Set<T> toSet(Optional<T> arg) {
+    return arg.isPresent() ? ImmutableSet.of(arg.get()) : ImmutableSet.<T>of();
+  }
+
+  static <ContentType> List<String> compareSets(String message, Set<ContentType> base, Set<ContentType> other) {
+    List<String> diffs = new ArrayList<>(2);
+    Set<ContentType> missingItems = Sets.difference(base, other);
+    if (!missingItems.isEmpty()) {
+      diffs.add(message + " Missing items: " + Joiner.on(", ").join(missingItems));
+    }
+    Set<ContentType> extraItems = Sets.difference(other, base);
+    if (!extraItems.isEmpty()) {
+      diffs.add(message + " Extra items: " + Joiner.on(", ").join(extraItems));
+    }
+    return diffs;
+  }
+
+
+  static <ContentType> List<String> compareConstraints(String tableName,
+                                             String constraintType,
+                                             Set<? extends Constraint<ContentType>> base,
+                                             Set<? extends Constraint<ContentType>> other,
+                                             boolean compareConstraintNames) {
+    List<String> diffs = new ArrayList<>();
+    Map<ContentType, Constraint<ContentType>> baseByContent = Maps.newHashMap();
+    Map<ContentType, Constraint<ContentType>> otherByContent = Maps.newHashMap();
+    for (Constraint<ContentType> c: base) {
+      baseByContent.put(c.content(), c);
+    }
+    for (Constraint<ContentType> c: other) {
+      otherByContent.put(c.content(), c);
+    }
+    diffs.addAll(compareSets(String.format("Comparing %ss of table %s.", constraintType, tableName),
+        baseByContent.keySet(),
+        otherByContent.keySet()));
+
+    Set<ContentType> common = Sets.intersection(baseByContent.keySet(), otherByContent.keySet());
+    for (ContentType constrContent : common) {
+      Constraint b = baseByContent.get(constrContent);
+      Constraint o = otherByContent.get(constrContent);
+      if (!b.name().equals(o.name())) {
+        if (compareConstraintNames) {
+          diffs.add(String.format("Constraint name mismatch for table %s: %s vs. %s", tableName, b, o));
+        }
+        else {
+          LOG.info("Ignoring constraint name mismatch for table {}: {} vs. {}", tableName, b, o);
+        }
+      }
+    }
+
+    return diffs;
+  }
+
+}

Nem az összes módosított fájl került megjelenítésre, mert túl sok fájl változott