You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2023/04/20 04:09:51 UTC

[calcite] branch main updated: [CALCITE-111] Support CONVERT function, for changing character sets

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

jhyde pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/main by this push:
     new b2917b33e8 [CALCITE-111] Support CONVERT function, for changing character sets
b2917b33e8 is described below

commit b2917b33e8dda434d2866d8a327f8a48520afb48
Author: ILuffZhe <il...@163.com>
AuthorDate: Sat Jan 29 15:02:41 2022 +0800

    [CALCITE-111] Support CONVERT function, for changing character sets
    
    This function is similar to the CONVERT as defined by the SQL
    standard, and the same as the CONVERT function in MySQL.
    
    It is different to MSSQL's CONVERT, which changes data types
    (like CAST), added in [CALCITE-5548].
    
    Close apache/calcite#2705
---
 core/src/main/codegen/templates/Parser.jj          |  60 ++++++-----
 .../calcite/adapter/enumerable/RexImpTable.java    |  22 ++++
 .../apache/calcite/runtime/CalciteResource.java    |   3 +
 .../org/apache/calcite/runtime/SqlFunctions.java   |  20 ++++
 .../main/java/org/apache/calcite/sql/SqlKind.java  |  12 ++-
 .../apache/calcite/sql/fun/SqlConvertFunction.java | 115 +++++++++++++++++----
 .../calcite/sql/fun/SqlStdOperatorTable.java       |  10 ++
 .../calcite/sql/validate/SqlValidatorImpl.java     |   7 +-
 .../calcite/sql2rel/StandardConvertletTable.java   |  14 +++
 .../org/apache/calcite/util/BuiltInMethod.java     |   2 +
 .../calcite/runtime/CalciteResource.properties     |   1 +
 .../java/org/apache/calcite/test/JdbcTest.java     |   7 --
 .../apache/calcite/test/SqlToRelConverterTest.java |   6 ++
 .../org/apache/calcite/test/SqlValidatorTest.java  |  10 ++
 .../apache/calcite/test/SqlToRelConverterTest.xml  |  12 +++
 core/src/test/resources/sql/functions.iq           |  62 ++++++++++-
 site/_docs/reference.md                            |   8 +-
 .../apache/calcite/sql/parser/SqlParserTest.java   |  17 ++-
 .../org/apache/calcite/test/SqlOperatorTest.java   |  15 +++
 19 files changed, 336 insertions(+), 67 deletions(-)

diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 33750d1f56..870de9dde2 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -6044,36 +6044,42 @@ SqlNode BuiltinFunctionCall() :
         <CONVERT> { s = span(); }
         <LPAREN>
         (
-          // CONVERT in the form of CONVERT(x USING y)
-
-          // "AddExpression" matches INTERVAL,
-          // which can also be 1st token in args of MSSQL CONVERT
-          // So lookahead another token (to match <USING> vs. <COMMA>)
-          LOOKAHEAD(2)
-          AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
-          <USING> name = SimpleIdentifier() { args.add(name); }
-          <RPAREN> {
-              return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args);
-          }
-        | // mssql CONVERT(type, val [,style])
-          (
-            dt = DataType() { args.add(dt); }
+            // CONVERT in the form of CONVERT(x USING y)
+
+            // "AddExpression" matches INTERVAL,
+            // which can also be 1st token in args of MSSQL CONVERT
+            // So lookahead another token (to match <USING> vs. <COMMA>)
+            LOOKAHEAD(2)
+            AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+            (
+                <USING> name = SimpleIdentifier() { args.add(name); }
             |
-            <INTERVAL> e = IntervalQualifier() { args.add(e); }
-          )
-          <COMMA>
-          AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
-          [
-            <COMMA>
+                <COMMA> e = SimpleIdentifier() { args.add(e); }
+                <COMMA> e = SimpleIdentifier() { args.add(e); }
+            )
+            <RPAREN> {
+                return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args);
+            }
+        |
+            // MSSql CONVERT(type, val [,style])
             (
-              style = UnsignedNumericLiteral() { args.add(style); }
-              |
-              <NULL> { args.add(SqlLiteral.createNull(getPos())); }
+                dt = DataType() { args.add(dt); }
+            |
+                <INTERVAL> e = IntervalQualifier() { args.add(e); }
             )
-          ]
-          <RPAREN> {
-            return SqlLibraryOperators.MSSQL_CONVERT.createCall(s.end(this), args);
-          }
+            <COMMA>
+            AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
+            [
+                <COMMA>
+                (
+                    style = UnsignedNumericLiteral() { args.add(style); }
+                |
+                    <NULL> { args.add(SqlLiteral.createNull(getPos())); }
+                )
+            ]
+            <RPAREN> {
+                return SqlLibraryOperators.MSSQL_CONVERT.createCall(s.end(this), args);
+            }
         )
     |
         <TRANSLATE> { s = span(); }
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 f3716f6bf4..02525cd38f 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
@@ -224,6 +224,7 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CLASSIFIER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COALESCE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COLLECT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CONCAT;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.CONVERT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COT;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.COUNT;
@@ -674,6 +675,7 @@ public class RexImpTable {
       map.put(TRY_CAST, new CastImplementor());
 
       map.put(REINTERPRET, new ReinterpretImplementor());
+      map.put(CONVERT, new ConvertImplementor());
 
       final RexCallImplementor value = new ValueConstructorImplementor();
       map.put(MAP_VALUE_CONSTRUCTOR, value);
@@ -3726,6 +3728,26 @@ public class RexImpTable {
     }
   }
 
+  /**
+   * Implementor for the {@code CONVERT} function.
+   *
+   * <p>If argument[0] is null, result is null.
+   */
+  private static class ConvertImplementor extends AbstractRexCallImplementor {
+    ConvertImplementor() {
+      super("convert", NullPolicy.STRICT, false);
+    }
+
+    @Override Expression implementSafe(RexToLixTranslator translator,
+        RexCall call, List<Expression> argValueList) {
+      final RexNode arg0 = call.getOperands().get(0);
+      if (SqlTypeUtil.isNull(arg0.getType())) {
+        return argValueList.get(0);
+      }
+      return Expressions.call(BuiltInMethod.CONVERT.method, argValueList);
+    }
+  }
+
   /**
    * Implementation that calls a given {@link java.lang.reflect.Method}.
    *
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 02a7b4aa49..4bf68b3771 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -502,6 +502,9 @@ public interface CalciteResource {
   @BaseMessage("Invalid type ''{0}'' in ORDER BY clause of ''{1}'' function. Only NUMERIC types are supported")
   ExInst<SqlValidatorException> unsupportedTypeInOrderBy(String a0, String a1);
 
+  @BaseMessage("Invalid type ''{0}'' in ''{1}'' function. Only ''{2}'' type is supported")
+  ExInst<SqlValidatorException> unsupportedTypeInConvertFunc(String a0, String a1, String a2);
+
   @BaseMessage("''{0}'' requires precisely one ORDER BY key")
   ExInst<SqlValidatorException> orderByRequiresOneKey(String a0);
 
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index a3c2ee8c04..cfd462ceaf 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -35,6 +35,7 @@ import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.rel.type.TimeFrame;
 import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.FlatLists.ComparableList;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlLibraryOperators;
 import org.apache.calcite.util.NumberUtil;
 import org.apache.calcite.util.TimeWithTimeZoneString;
@@ -59,6 +60,10 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.math.MathContext;
 import java.math.RoundingMode;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
 import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
@@ -748,6 +753,21 @@ public class SqlFunctions {
     return String.join("", args);
   }
 
+  /** SQL {@code CONVERT(s, src_charset, dest_charset)} function. */
+  public static String convertWithCharset(String s, String srcCharset,
+      String destCharset) {
+    final Charset src = SqlUtil.getCharset(srcCharset);
+    final Charset dest = SqlUtil.getCharset(destCharset);
+    byte[] bytes = s.getBytes(src);
+    final CharsetDecoder decoder = dest.newDecoder();
+    final ByteBuffer buffer = ByteBuffer.wrap(bytes);
+    try {
+      return decoder.decode(buffer).toString();
+    } catch (CharacterCodingException ex) {
+      throw RESOURCE.charsetEncoding(s, dest.name()).ex();
+    }
+  }
+
   /** SQL {@code RTRIM} function applied to string. */
   public static String rtrim(String s) {
     return trim(false, true, " ", s);
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlKind.java b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
index 4fc34072f2..e6f551fb3a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlKind.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlKind.java
@@ -144,6 +144,9 @@ public enum SqlKind {
    */
   SET_SEMANTICS_TABLE,
 
+  /** {@code CONVERT} function. */
+  CONVERT,
+
   /** POSITION function. */
   POSITION,
 
@@ -1203,6 +1206,7 @@ public enum SqlKind {
    * {@link #JOIN},
    * {@link #OTHER_FUNCTION},
    * {@link #CAST},
+   * {@link #CONVERT},
    * {@link #TRIM},
    * {@link #LITERAL_CHAIN},
    * {@link #JDBC_FN},
@@ -1217,7 +1221,7 @@ public enum SqlKind {
   public static final Set<SqlKind> EXPRESSION =
       EnumSet.complementOf(
           concat(
-              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, DEFAULT,
+              EnumSet.of(AS, ARGUMENT_ASSIGNMENT, CONVERT, DEFAULT,
                   RUNNING, FINAL, LAST, FIRST, PREV, NEXT,
                   FILTER, WITHIN_GROUP, IGNORE_NULLS, RESPECT_NULLS, SEPARATOR,
                   DESCENDING, CUBE, ROLLUP, GROUPING_SETS, EXTEND, LATERAL,
@@ -1242,10 +1246,12 @@ public enum SqlKind {
    * Category consisting of regular and special functions.
    *
    * <p>Consists of regular functions {@link #OTHER_FUNCTION} and special
-   * functions {@link #ROW}, {@link #TRIM}, {@link #CAST}, {@link #REVERSE}, {@link #JDBC_FN}.
+   * functions {@link #ROW}, {@link #TRIM}, {@link #CAST}, {@link #REVERSE},
+   * {@link #JDBC_FN}.
    */
   public static final Set<SqlKind> FUNCTION =
-      EnumSet.of(OTHER_FUNCTION, ROW, TRIM, LTRIM, RTRIM, CAST, REVERSE, JDBC_FN, POSITION);
+      EnumSet.of(OTHER_FUNCTION, ROW, TRIM, LTRIM, RTRIM, CAST, REVERSE,
+          JDBC_FN, POSITION, CONVERT);
 
   /**
    * Category of SqlAvgAggFunction.
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
index f36621858f..cd79137a38 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlConvertFunction.java
@@ -16,52 +16,127 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
-import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+import java.util.List;
+
+import static org.apache.calcite.util.Static.RESOURCE;
+
+import static java.util.Objects.requireNonNull;
 
 /**
- * Common base for the <code>CONVERT</code> and <code>TRANSLATE</code>
- * functions.
+ * Common base for the <code>CONVERT</code> function.
+ * <p>The SQL syntax is
+ *
+ * <blockquote><pre>
+ *   {@code CONVERT(characterString, sourceCharset, destCharset)}
+ * </pre></blockquote>
  */
 public class SqlConvertFunction extends SqlFunction {
   //~ Constructors -----------------------------------------------------------
 
   protected SqlConvertFunction(String name) {
-    super(
-        name,
-        SqlKind.OTHER_FUNCTION,
-        ReturnTypes.ARG0,
-        null,
-        OperandTypes.ANY_ANY,
+    super(name, SqlKind.CONVERT, ReturnTypes.ARG0, null, null,
         SqlFunctionCategory.STRING);
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override public void unparse(
-      SqlWriter writer,
-      SqlCall call,
-      int leftPrec,
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    // The base method validates all operands. We override because
+    // we don't want to validate the Charset as identifier.
+    final List<SqlNode> operands = call.getOperandList();
+    assert operands.size() == 3;
+    operands.get(0).validateExpr(validator, scope);
+    // validate if the Charsets are legal.
+    assert operands.get(1) instanceof SqlIdentifier;
+    final String src_charset = operands.get(1).toString();
+    SqlUtil.getCharset(src_charset);
+    assert operands.get(2) instanceof SqlIdentifier;
+    final String dest_charset = operands.get(2).toString();
+    SqlUtil.getCharset(dest_charset);
+    super.validateQuantifier(validator, call);
+  }
+
+  @Override public <R> void acceptCall(SqlVisitor<R> visitor, SqlCall call,
+      boolean onlyExpressions, SqlBasicVisitor.ArgHandler<R> argHandler) {
+    if (onlyExpressions) {
+      // Both operand[1] and operand[2] are not an expression, but Charset
+      // identifier
+      argHandler.visitChild(visitor, call, 0, call.operand(0));
+    } else {
+      super.acceptCall(visitor, call, onlyExpressions, argHandler);
+    }
+  }
+
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    // special case for CONVERT: don't need to derive type for Charsets
+    RelDataType nodeType =
+        validator.deriveType(scope, call.operand(0));
+    requireNonNull(nodeType, "nodeType");
+    return validateOperands(validator, scope, call);
+  }
+
+  @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+      boolean throwOnFailure) {
+    // type of operand[0] should be Character or NULL
+    final RelDataType t = callBinding.getOperandType(0);
+    if (SqlTypeUtil.isNull(t)) {
+      // convert(null, src_charset, dest_charset) is supported
+      return true;
+    }
+    if (!SqlTypeUtil.inCharFamily(t)) {
+      if (throwOnFailure) {
+        throw callBinding.newValidationError(
+            RESOURCE.unsupportedTypeInConvertFunc(t.getFullTypeString(),
+                "CONVERT", "CHARACTER"));
+      }
+      return false;
+    }
+    return true;
+  }
+
+  @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
       int rightPrec) {
     final SqlWriter.Frame frame = writer.startFunCall(getName());
-    call.operand(0).unparse(writer, leftPrec, rightPrec);
-    writer.sep("USING");
-    call.operand(1).unparse(writer, leftPrec, rightPrec);
+    for (SqlNode node : call.getOperandList()) {
+      writer.sep(",");
+      node.unparse(writer, leftPrec, rightPrec);
+    }
     writer.endFunCall(frame);
   }
 
   @Override public String getSignatureTemplate(final int operandsCount) {
+    //noinspection SwitchStatementWithTooFewBranches
     switch (operandsCount) {
-    case 2:
-      return "{0}({1} USING {2})";
+    case 3:
+      return "{0}({1}, {2}, {3})";
     default:
-      break;
+      throw new IllegalStateException("operandsCount should be 3, got "
+          + operandsCount);
     }
-    throw new IllegalStateException("operandsCount should be 2, got " + operandsCount);
+  }
+
+  @Override public SqlOperandCountRange getOperandCountRange() {
+    return SqlOperandCountRanges.of(3);
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index e1427ab35b..44bdf44a3e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -1575,6 +1575,16 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
       SqlBasicFunction.create("REPLACE", ReturnTypes.ARG0_NULLABLE_VARYING,
           OperandTypes.STRING_STRING_STRING, SqlFunctionCategory.STRING);
 
+  /** The {@code CONVERT(charValue, srcCharsetName, destCharsetName)}
+   * function converts {@code charValue} with {@code destCharsetName},
+   * whose original encoding is specified by {@code srcCharsetName}.
+   *
+   * <p>The SQL standard defines
+   * {@code CONVERT(charValue USING transcodingName)}, and MySQL implements it;
+   * Calcite does not currently support this.
+   *
+   * <p>MySQL and Microsoft SQL Server have a {@code CONVERT(type, value)}
+   * function; Calcite does not currently support this, either. */
   public static final SqlFunction CONVERT =
       new SqlConvertFunction("CONVERT");
 
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 941733175a..3a28b178ba 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
@@ -3868,8 +3868,13 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         // we stripped the field access. Recurse to this method, the DOT's operand
         // can be another SqlCall, or an SqlIdentifier.
         checkRollUp(grandParent, parent, stripDot, scope, contextClause);
+      } else if (stripDot.getKind() == SqlKind.CONVERT) {
+        // operand[1] doesn't need to be checked for CONVERT
+        SqlNode child = ((SqlCall) stripDot).getOperandList().get(0);
+        checkRollUp(parent, current, child, scope, contextClause);
       } else {
-        List<? extends @Nullable SqlNode> children = ((SqlCall) stripDot).getOperandList();
+        List<? extends @Nullable SqlNode> children =
+            ((SqlCall) stripDot).getOperandList();
         for (SqlNode child : children) {
           checkRollUp(parent, current, child, scope, contextClause);
         }
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 926427713d..fae2a1ea8b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -264,6 +264,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     // "AS" has no effect, so expand "x AS id" into "x".
     registerOp(SqlStdOperatorTable.AS,
         (cx, call) -> cx.convertExpression(call.operand(0)));
+    registerOp(SqlStdOperatorTable.CONVERT, this::convertCharset);
     // "SQRT(x)" is equivalent to "POWER(x, .5)"
     registerOp(SqlStdOperatorTable.SQRT,
         (cx, call) -> cx.convertExpression(
@@ -741,6 +742,19 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     return convertFunction(cx, (SqlFunction) call.getOperator(), call);
   }
 
+  protected RexNode convertCharset(
+      @UnknownInitialization StandardConvertletTable this,
+      SqlRexContext cx, SqlCall call) {
+    final SqlNode expr = call.operand(0);
+    final String srcCharset = call.operand(1).toString();
+    final String destCharset = call.operand(2).toString();
+    final RexBuilder rexBuilder = cx.getRexBuilder();
+    return rexBuilder.makeCall(SqlStdOperatorTable.CONVERT,
+        cx.convertExpression(expr),
+        rexBuilder.makeLiteral(srcCharset),
+        rexBuilder.makeLiteral(destCharset));
+  }
+
   /**
    * Converts a call to the {@code EXTRACT} function.
    *
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index bab8c1255f..251cdcfce3 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -347,6 +347,8 @@ public enum BuiltInMethod {
   ASCII(SqlFunctions.class, "ascii", String.class),
   CHAR_FROM_ASCII(SqlFunctions.class, "charFromAscii", int.class),
   CHAR_FROM_UTF8(SqlFunctions.class, "charFromUtf8", int.class),
+  CONVERT(SqlFunctions.class, "convertWithCharset", String.class, String.class,
+      String.class),
   REPEAT(SqlFunctions.class, "repeat", String.class, int.class),
   SPACE(SqlFunctions.class, "space", int.class),
   SOUNDEX(SqlFunctions.class, "soundex", String.class),
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 ad0acc822b..5c604ca60a 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -169,6 +169,7 @@ OrderByOverlap=ORDER BY not allowed in both base and referenced windows
 RefWindowWithFrame=Referenced window cannot have framing declarations
 TypeNotSupported=Type ''{0}'' is not supported
 UnsupportedTypeInOrderBy=Invalid type ''{0}'' in ORDER BY clause of ''{1}'' function. Only NUMERIC types are supported
+UnsupportedTypeInConvertFunc=Invalid type ''{0}'' in ''{1}'' function. Only ''{2}'' type is supported
 OrderByRequiresOneKey=''{0}'' requires precisely one ORDER BY key
 FunctionQuantifierNotAllowed=DISTINCT/ALL not allowed with {0} function
 WithinGroupNotAllowed=WITHIN GROUP not allowed with {0} function
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index e62d455277..4a5425bf4a 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -6694,13 +6694,6 @@ public class JdbcTest {
         "select * from \"employee\" where \"full_name\" = _UTF16'\u82f1\u56fd'")
         .throws_(
             "Cannot apply = to the two different charsets ISO-8859-1 and UTF-16LE");
-
-    // The CONVERT function (what SQL:2011 calls "character transliteration") is
-    // not implemented yet. See
-    // https://issues.apache.org/jira/browse/CALCITE-111.
-    with.query("select * from \"employee\"\n"
-        + "where convert(\"full_name\" using UTF16) = _UTF16'\u82f1\u56fd'")
-        .throws_("Column 'UTF16' not found in any table");
   }
 
   /** Tests metadata for the MySQL lexical scheme. */
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 7541d68e27..3c863ba9b8 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -3156,6 +3156,12 @@ class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).withExpand(false).ok();
   }
 
+  @Test void testConvertFunc() {
+    final String sql = "select convert(ename, latin1, utf8) as new_ename\n"
+        + "from emp";
+    sql(sql).withTrim(true).ok();
+  }
+
   /**
    * Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-695">[CALCITE-695]
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index bc071f6827..b49a7632fd 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7292,6 +7292,16 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
             + "'BOOL_OR\\(<VARCHAR\\(20\\)>\\)'.*");
   }
 
+  @Test void testConvertFunction() {
+    sql("select convert(ename, utf16, utf8) from emp").ok();
+    sql("select convert(cast(deptno as varchar), utf16, utf8) from emp");
+    sql("select convert(null, gbk, utf8) from emp");
+    sql("select ^convert(deptno, utf8, latin1)^ from emp")
+        .fails("Invalid type 'INTEGER NOT NULL' in 'CONVERT' function\\. "
+            + "Only 'CHARACTER' type is supported");
+    sql("select convert(ename, utf8, utf9) from emp").fails("UTF9");
+  }
+
   @Test void testFunctionalDistinct() {
     sql("select count(distinct sal) from emp").ok();
     sql("select COALESCE(^distinct^ sal) from emp")
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 25c1a3c302..44405c16d8 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -692,6 +692,18 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
   LogicalJoin(condition=[=($9, $7)], joinType=[inner])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testConvertFunc">
+    <Resource name="sql">
+      <![CDATA[select convert(ename, latin1, utf8) as new_ename
+from emp]]>
+    </Resource>
+    <Resource name="plan">
+      <![CDATA[
+LogicalProject(NEW_ENAME=[CONVERT($1, 'LATIN1', 'UTF8')])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
     </Resource>
   </TestCase>
diff --git a/core/src/test/resources/sql/functions.iq b/core/src/test/resources/sql/functions.iq
index 9f5d6b02b1..309ebf9f5f 100644
--- a/core/src/test/resources/sql/functions.iq
+++ b/core/src/test/resources/sql/functions.iq
@@ -90,6 +90,61 @@ SELECT COMPRESS('sample');
 
 !ok
 
+# -----------------------------------------------------------------------------
+# The standard CONVERT function, for changing character sets.
+# (The MSSQL CONVERT function, equivalent to CAST, appears later in this file.)
+
+select convert('abcd', latin1, utf8);
++--------+
+| EXPR$0 |
++--------+
+| abcd   |
++--------+
+(1 row)
+
+!ok
+
+select convert(null, latin1, utf8);
++--------+
+| EXPR$0 |
++--------+
+|        |
++--------+
+(1 row)
+
+!ok
+
+!use foodmart
+select convert(cast("employee_id" as varchar), utf8, latin1) as alia
+from "employee"
+limit 3;
++------+
+| ALIA |
++------+
+| 1    |
+| 2    |
+| 4    |
++------+
+(3 rows)
+
+!ok
+
+select "employee_id"
+from "employee"
+where convert(cast("employee_id" as varchar), utf8, latin1) <> 1
+limit 3;
++-------------+
+| employee_id |
++-------------+
+|           2 |
+|           4 |
+|           5 |
++-------------+
+(3 rows)
+
+!ok
+
+# -----------------------------------------------------------------------------
 
 !use oraclefunc
 
@@ -219,6 +274,7 @@ EnumerableCalc(expr#0..4=[{inputs}], expr#5=[IS NOT NULL($t2)], expr#6=[CAST($t2
 
 !ok
 
+# -----------------------------------------------------------------------------
 # [CALCITE-5548] - Add support for MSSQL CONVERT function
 !use mssqlfunc
 select CONVERT(INTEGER, 45.4);
@@ -300,7 +356,9 @@ select CONVERT(INTEGER, NULL, NULL);
 # below not executed due to huge stack trace quidem insists on generating
 # unsupported format test for documentation (dd/mm/yyyy) (need MSSQL "style" operand support in enumerable)
 # however CONVERT right now is simply a wrapper around CAST
-# select CONVERT(DATE, '05/01/2000', 103);
-
+!if (false) {
+select CONVERT(DATE, '05/01/2000', 103);
+!ok
+!}
 
 # End functions.iq
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index a2c837639e..5507fbe476 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -27,7 +27,6 @@ here to appease testAllFunctionsAreDocumented:
 |:-------------- |:---------------------
 | CALL           | TODO: document
 | CLASSIFIER()   | Documented with MATCH_RECOGNIZE
-| CONVERT()      | In SqlStdOperatorTable, but not fully implemented
 | CUME_DIST()    | In SqlStdOperatorTable, but not fully implemented
 | DESC           | Described as part of ORDER BY syntax
 | EQUALS         | Documented as an period operator
@@ -1463,9 +1462,10 @@ Algorithms for implicit conversion are subject to change across Calcite releases
 
 #### Explicit Type Conversion
 
-| Operator syntax | Description
-|:--------------- | :----------
-| CAST(value AS type) | Converts a value to a given type.
+| Operator syntax                     | Description
+|:------------------------------------| :----------
+| CAST(value AS type)                 | Converts a value to a given type
+| CONVERT(string, charSet1, charSet2) | Converts *string* from *charSet1* to *charSet2*
 
 Supported data types syntax:
 
diff --git a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 8468edf79d..f854df66eb 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -5348,10 +5348,21 @@ public class SqlParserTest {
   }
 
   @Test void testConvertAndTranslate() {
-    expr("convert('abc' using conversion)")
-        .ok("CONVERT('abc' USING `CONVERSION`)");
+    expr("convert('abc', utf8, utf16)")
+        .ok("CONVERT('abc', `UTF8`, `UTF16`)");
+    sql("select convert(name, latin1, gbk) as newName from t")
+            .ok("SELECT CONVERT(`NAME`, `LATIN1`, `GBK`) AS `NEWNAME`\n"
+              + "FROM `T`");
+
+    // CONVERT function in Mysql
+    if (false) {
+      expr("convert('abc' using conversion)")
+          .ok("CONVERT('abc' USING `CONVERSION`)");
+    }
+
+    // TRANSLATE need to be implemented (syntax may be different)
     expr("translate('abc' using lazy_translation)")
-        .ok("TRANSLATE('abc' USING `LAZY_TRANSLATION`)");
+        .ok("TRANSLATE('abc', `LAZY_TRANSLATION`)");
   }
 
   @Test void testTranslate3() {
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
index 12ae35ee29..9d4c2a424a 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -3574,6 +3574,21 @@ public class SqlOperatorTest {
   @Test void testConvertFunc() {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.CONVERT, VM_FENNEL, VM_JAVA);
+    f.checkFails("convert('a', utf8, utf10)", "UTF10", false);
+    f.checkFails("select ^convert(col, latin1, utf8)^\n"
+            + "from (select 1 as col\n"
+            + " from (values(true)))",
+        "Invalid type 'INTEGER NOT NULL' in 'CONVERT' function\\. "
+            + "Only 'CHARACTER' type is supported",
+        false);
+    f.check("select convert(col, latin1, utf8)\n"
+            + "from (select 'a' as col\n"
+            + " from (values(true)))",
+        SqlTests.ANY_TYPE_CHECKER, 'a');
+
+    f.checkType("convert('a', utf16, gbk)", "CHAR(1) NOT NULL");
+    f.checkType("convert(null, utf16, gbk)", "NULL");
+    f.checkType("convert(cast(1 as varchar(2)), utf8, latin1)", "VARCHAR(2) NOT NULL");
   }
 
   @Test void testTranslateFunc() {