You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ma...@apache.org on 2015/12/09 04:06:45 UTC

calcite git commit: [CALCITE-992] Validate and resolve sequence reference as a Table object

Repository: calcite
Updated Branches:
  refs/heads/master 937fc461a -> de3880298


[CALCITE-992] Validate and resolve sequence reference as a Table object


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

Branch: refs/heads/master
Commit: de3880298b180a38013c2a748758c863082ac272
Parents: 937fc46
Author: maryannxue <we...@intel.com>
Authored: Tue Dec 8 22:03:13 2015 -0500
Committer: maryannxue <we...@intel.com>
Committed: Tue Dec 8 22:06:06 2015 -0500

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java | 44 ++++++++++++-----
 .../calcite/prepare/CalcitePrepareImpl.java     |  9 +++-
 .../org/apache/calcite/prepare/Prepare.java     |  2 +
 .../apache/calcite/runtime/CalciteResource.java |  3 ++
 .../sql/fun/SqlSequenceValueOperator.java       |  9 ++++
 .../calcite/sql/validate/SqlValidator.java      |  2 +
 .../calcite/sql/validate/SqlValidatorImpl.java  | 24 +++++++++
 .../sql2rel/StandardConvertletTable.java        |  3 +-
 .../main/java/org/apache/calcite/util/Util.java | 52 ++++++++++++++++++++
 .../calcite/runtime/CalciteResource.properties  |  1 +
 .../java/org/apache/calcite/util/UtilTest.java  | 20 ++++++++
 core/src/test/resources/sql/sequence.oq         | 27 +++++++++-
 12 files changed, 178 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 07f3f43..dd21b9d 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -31,6 +31,8 @@ import org.apache.calcite.linq4j.tree.OptimizeVisitor;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.linq4j.tree.Types;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
@@ -188,8 +190,7 @@ public class RexImpTable {
   public static final MemberExpression BOXED_TRUE_EXPR =
       Expressions.field(null, Boolean.class, "TRUE");
 
-  private final Map<SqlOperator, CallImplementor> map =
-      new HashMap<SqlOperator, CallImplementor>();
+  private final Map<SqlOperator, CallImplementor> map = new HashMap<>();
   private final Map<SqlAggFunction, Supplier<? extends AggImplementor>> aggMap =
       Maps.newHashMap();
   private final Map<SqlAggFunction, Supplier<? extends WinAggImplementor>>
@@ -304,10 +305,10 @@ public class RexImpTable {
 
     // Sequences
     defineImplementor(CURRENT_VALUE, NullPolicy.STRICT,
-        new MethodImplementor(BuiltInMethod.SEQUENCE_CURRENT_VALUE.method),
+        new SequenceImplementor(BuiltInMethod.SEQUENCE_CURRENT_VALUE.method),
         false);
     defineImplementor(NEXT_VALUE, NullPolicy.STRICT,
-        new MethodImplementor(BuiltInMethod.SEQUENCE_NEXT_VALUE.method),
+        new SequenceImplementor(BuiltInMethod.SEQUENCE_NEXT_VALUE.method),
         false);
 
     // System functions
@@ -363,10 +364,7 @@ public class RexImpTable {
         } catch (InstantiationException e) {
           throw new IllegalStateException(
               "Unable to instantiate aggregate implementor " + constructor, e);
-        } catch (IllegalAccessException e) {
-          throw new IllegalStateException(
-              "Error while creating aggregate implementor " + constructor, e);
-        } catch (InvocationTargetException e) {
+        } catch (IllegalAccessException | InvocationTargetException e) {
           throw new IllegalStateException(
               "Error while creating aggregate implementor " + constructor, e);
         }
@@ -646,7 +644,7 @@ public class RexImpTable {
   private static List<RexNode> harmonize(
       final RexToLixTranslator translator, final List<RexNode> operands) {
     int nullCount = 0;
-    final List<RelDataType> types = new ArrayList<RelDataType>();
+    final List<RelDataType> types = new ArrayList<>();
     final RelDataTypeFactory typeFactory =
         translator.builder.getTypeFactory();
     for (RexNode operand : operands) {
@@ -672,7 +670,7 @@ public class RexImpTable {
       return operands;
     }
     assert (nullCount > 0) == type.isNullable();
-    final List<RexNode> list = new ArrayList<RexNode>();
+    final List<RexNode> list = new ArrayList<>();
     for (RexNode operand : operands) {
       list.add(
           translator.builder.ensureType(type, operand, false));
@@ -755,7 +753,7 @@ public class RexImpTable {
       NullAs nullAs,
       NullPolicy nullPolicy,
       NotNullImplementor implementor) {
-    final List<Expression> list = new ArrayList<Expression>();
+    final List<Expression> list = new ArrayList<>();
     switch (nullAs) {
     case NULL:
       // v0 == null || v1 == null ? null : f(v0, v1)
@@ -806,7 +804,7 @@ public class RexImpTable {
       // The cases with setNullable above might not help since the same
       // RexNode can be referred via multiple ways: RexNode itself, RexLocalRef,
       // and may be others.
-      Map<RexNode, Boolean> nullable = new HashMap<RexNode, Boolean>();
+      final Map<RexNode, Boolean> nullable = new HashMap<>();
       if (nullPolicy == NullPolicy.STRICT) {
         // The arguments should be not nullable if STRICT operator is computed
         // in nulls NOT_POSSIBLE mode
@@ -1134,7 +1132,7 @@ public class RexImpTable {
         AggAddContext add) {
       List<Expression> acc = add.accumulator();
       List<Expression> aggArgs = add.arguments();
-      List<Expression> args = new ArrayList<Expression>(aggArgs.size() + 1);
+      List<Expression> args = new ArrayList<>(aggArgs.size() + 1);
       args.add(acc.get(0));
       args.addAll(aggArgs);
       add.currentBlock().add(
@@ -1518,6 +1516,26 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for a function that generates calls to a given method. */
+  private static class SequenceImplementor extends MethodImplementor {
+    SequenceImplementor(Method method) {
+      super(method);
+    }
+
+    public Expression implement(
+        RexToLixTranslator translator,
+        RexCall call,
+        List<Expression> translatedOperands) {
+      assert translatedOperands.size() == 1;
+      ConstantExpression x = (ConstantExpression) translatedOperands.get(0);
+      List<String> names = Util.stringToList((String) x.value);
+      RelOptTable table =
+          Prepare.CatalogReader.THREAD_LOCAL.get().getTable(names);
+      System.out.println("Now, do something with table " + table);
+      return super.implement(translator, call, translatedOperands);
+    }
+  }
+
   /** Implementor for SQL functions that generates calls to a given method name.
    *
    * <p>Use this, as opposed to {@link MethodImplementor}, if the SQL function

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 6d7fb31..db4de78 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -1172,8 +1172,13 @@ public class CalcitePrepareImpl implements CalcitePrepare {
           enumerable = EnumerableCalc.create(enumerable, program);
         }
 
-        bindable = EnumerableInterpretable.toBindable(internalParameters,
-            context.spark(), enumerable, prefer);
+        try {
+          CatalogReader.THREAD_LOCAL.set(catalogReader);
+          bindable = EnumerableInterpretable.toBindable(internalParameters,
+              context.spark(), enumerable, prefer);
+        } finally {
+          CatalogReader.THREAD_LOCAL.remove();
+        }
       }
 
       if (timingTracer != null) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index 399a178..4d3fc4b 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -382,6 +382,8 @@ public abstract class Prepare {
     CatalogReader withSchemaPath(List<String> schemaPath);
 
     PreparingTable getTable(List<String> names);
+
+    ThreadLocal<CatalogReader> THREAD_LOCAL = new ThreadLocal<>();
   }
 
   /** Definition of a table, for the purposes of the validator and planner. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 6eabafd..8c93081 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -152,6 +152,9 @@ public interface CalciteResource {
   @BaseMessage("Table ''{0}'' not found")
   ExInst<SqlValidatorException> tableNameNotFound(String a0);
 
+  @BaseMessage("Table ''{0}'' is not a sequence")
+  ExInst<SqlValidatorException> notASequence(String a0);
+
   @BaseMessage("Column ''{0}'' not found in any table")
   ExInst<SqlValidatorException> columnNotFound(String a0);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
index 73b224e..357716d 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlSequenceValueOperator.java
@@ -19,13 +19,17 @@ package org.apache.calcite.sql.fun;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import java.util.List;
+
 /** Operator that returns the current or next value of a sequence. */
 public class SqlSequenceValueOperator extends SqlSpecialOperator {
   /** Creates a SqlSequenceValueOperator. */
@@ -54,6 +58,11 @@ public class SqlSequenceValueOperator extends SqlSpecialOperator {
 
   @Override public void validateCall(SqlCall call, SqlValidator validator,
       SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    List<SqlNode> operands = call.getOperandList();
+    assert operands.size() == 1;
+    assert operands.get(0) instanceof SqlIdentifier;
+    SqlIdentifier id = (SqlIdentifier) operands.get(0);
+    validator.validateSequenceValue(scope, id);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index 7878917..1adc9d8 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -740,6 +740,8 @@ public interface SqlValidator {
 
   void validateWithItem(SqlWithItem withItem);
 
+  void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id);
+
   SqlValidatorScope getWithScope(SqlNode withItem);
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 1218d5a..8b92b91 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -27,6 +28,7 @@ import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Feature;
 import org.apache.calcite.runtime.Resources;
+import org.apache.calcite.schema.Table;
 import org.apache.calcite.sql.JoinConditionType;
 import org.apache.calcite.sql.JoinType;
 import org.apache.calcite.sql.SqlAccessEnum;
@@ -3206,6 +3208,28 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
+  public void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id) {
+    // Resolve identifier as a table.
+    final SqlValidatorNamespace ns = scope.getTableNamespace(id.names);
+    if (ns == null) {
+      throw newValidationError(id, RESOURCE.tableNameNotFound(id.toString()));
+    }
+
+    // We've found a table. But is it a sequence?
+    if (!(ns instanceof TableNamespace)) {
+      throw newValidationError(id, RESOURCE.notASequence(id.toString()));
+    }
+    final SqlValidatorTable table = ns.getTable();
+    final Table table1 = ((RelOptTable) table).unwrap(Table.class);
+    switch (table1.getJdbcTableType()) {
+    case SEQUENCE:
+    case TEMPORARY_SEQUENCE:
+      break;
+    default:
+      throw newValidationError(id, RESOURCE.notASequence(id.toString()));
+    }
+  }
+
   public SqlValidatorScope getWithScope(SqlNode withItem) {
     assert withItem.getKind() == SqlKind.WITH_ITEM;
     return scopes.get(withItem);

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index b184982..af34603 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -695,7 +695,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     final List<SqlNode> operands = call.getOperandList();
     assert operands.size() == 1;
     assert operands.get(0) instanceof SqlIdentifier;
-    String key = ((SqlIdentifier) operands.get(0)).names.toString();
+    final SqlIdentifier id = (SqlIdentifier) operands.get(0);
+    final String key = Util.listToString(id.names);
     RelDataType returnType =
         cx.getValidator().getValidatedNodeType(call);
     return cx.getRexBuilder().makeCall(returnType, fun,

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index eea3e47..f7db6a4 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2120,6 +2120,58 @@ public class Util {
     return true;
   }
 
+  /** Converts ["ab", "c"] to "ab"."c". */
+  public static String listToString(List<String> list) {
+    final StringBuilder b = new StringBuilder();
+    for (String s : list) {
+      if (b.length() > 0) {
+        b.append(".");
+      }
+      b.append('"');
+      b.append(s.replace("\"", "\"\""));
+      b.append('"');
+    }
+    return b.toString();
+  }
+
+  public static List<String> stringToList(String s) {
+    if (s.isEmpty()) {
+      return ImmutableList.of();
+    }
+    final ImmutableList.Builder<String> builder = ImmutableList.builder();
+    final StringBuilder b = new StringBuilder();
+    int i = 0;
+    for (;;) {
+      char c = s.charAt(i);
+      if (c != '"') {
+        throw new IllegalArgumentException();
+      }
+      for (;;) {
+        c = s.charAt(++i);
+        if (c == '"') {
+          if (i == s.length() - 1) {
+            break;
+          }
+          ++i;
+          c = s.charAt(i);
+          if (c == '.') {
+            break;
+          }
+          if (c != '"') {
+            throw new IllegalArgumentException();
+          }
+        }
+        b.append(c);
+      }
+      builder.add(b.toString());
+      b.setLength(0);
+      if (++i >= s.length()) {
+        break;
+      }
+    }
+    return builder.build();
+  }
+
   /** Converts a number into human-readable form, with 3 digits and a "K", "M"
    * or "G" multiplier for thousands, millions or billions.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 7e97a51..85dcc1d 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -55,6 +55,7 @@ DuplicateTargetColumn=Target column ''{0}'' is assigned more than once
 UnmatchInsertColumn=Number of INSERT target columns ({0,number}) does not equal number of source items ({1,number})
 TypeNotAssignable=Cannot assign to target field ''{0}'' of type {1} from source field ''{2}'' of type {3}
 TableNameNotFound=Table ''{0}'' not found
+NotASequence=Table ''{0}'' is not a sequence
 ColumnNotFound=Column ''{0}'' not found in any table
 ColumnNotFoundInTable=Column ''{0}'' not found in table ''{1}''
 ColumnAmbiguous=Column ''{0}'' is ambiguous

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 56336ce..9a42fcc 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -1427,6 +1427,26 @@ public class UtilTest {
   /** Dummy method for {@link #testParameterName()} to inspect. */
   public static void foo(int i, @Parameter(name = "j") int j) {
   }
+
+  @Test public void testListToString() {
+    checkListToString("x");
+    checkListToString("");
+    checkListToString();
+    checkListToString("ab", "c", "");
+    checkListToString("ab", "c", "", "de");
+    checkListToString("ab", "c.");
+    checkListToString("ab", "c.d");
+    checkListToString("ab", ".d");
+    checkListToString(".ab", "d");
+    checkListToString(".a", "d");
+    checkListToString("a.", "d");
+  }
+
+  private void checkListToString(String... strings) {
+    final List<String> list = ImmutableList.copyOf(strings);
+    final String asString = Util.listToString(list);
+    assertThat(Util.stringToList(asString), is(list));
+  }
 }
 
 // End UtilTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/de388029/core/src/test/resources/sql/sequence.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/sequence.oq b/core/src/test/resources/sql/sequence.oq
index cf18122..4f338de 100644
--- a/core/src/test/resources/sql/sequence.oq
+++ b/core/src/test/resources/sql/sequence.oq
@@ -19,7 +19,7 @@
 !use seq
 !set outputformat mysql
 
-select next value for my_seq as c from (values 1, 2);
+select next value for "my_seq" as c from (values 1, 2);
 +---+
 | C |
 +---+
@@ -29,7 +29,7 @@ select next value for my_seq as c from (values 1, 2);
 (2 rows)
 
 !ok
-select current value for my_seq as c from (values 1, 2);
+select current value for "my_seq" as c from (values 1, 2);
 +---+
 | C |
 +---+
@@ -40,6 +40,29 @@ select current value for my_seq as c from (values 1, 2);
 
 !ok
 
+select next value for "my_seq" as c from (values 1, 2);
+C BIGINT(19) NOT NULL
+!type
+
+# Qualified with schema name
+select next value for "s"."my_seq" as c from (values 1, 2);
+C BIGINT(19) NOT NULL
+!type
+
+select next value for "unknown_seq" as c from (values 1, 2);
+From line 1, column 23 to line 1, column 35: Table 'unknown_seq' not found
+!error
+
+# Qualified with bad schema name
+select next value for "unknown_schema"."my_seq" as c from (values 1, 2);
+From line 1, column 23 to line 1, column 47: Table 'unknown_schema.my_seq' not found
+!error
+
+# Table found, but not a sequence
+select next value for "metadata".tables as c from (values 1, 2);
+From line 1, column 23 to line 1, column 39: Table 'metadata.TABLES' is not a sequence
+!error
+
 # Sequences appear in the catalog as tables of type 'SEQUENCE'
 select * from "metadata".tables;
 +----------+------------+-----------+--------------+---------+---------+-----------+----------+------------------------+---------------+