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 2019/07/10 03:38:06 UTC

[calcite] branch master updated: [CALCITE-2995] Implement DAYNAME,MONTHNAME functions; add "locale" connection property (xuqianjin)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 72c52f4  [CALCITE-2995] Implement DAYNAME,MONTHNAME functions; add "locale" connection property (xuqianjin)
72c52f4 is described below

commit 72c52f46eab7fbe57ae5236ecc3831113684ff71
Author: XuQianJin-Stars <x1...@163.com>
AuthorDate: Sun Apr 14 15:24:25 2019 +0800

    [CALCITE-2995] Implement DAYNAME,MONTHNAME functions; add "locale" connection property (xuqianjin)
    
    Add locale connection property (Julian Hyde).
    
    Close apache/calcite#1163
---
 .../main/java/org/apache/calcite/DataContext.java  |   7 +
 .../calcite/adapter/enumerable/RexImpTable.java    |  82 +++++++--
 .../calcite/config/CalciteConnectionConfig.java    |   2 +
 .../config/CalciteConnectionConfigImpl.java        |   5 +
 .../calcite/config/CalciteConnectionProperty.java  |   7 +
 .../apache/calcite/jdbc/CalciteConnectionImpl.java |  12 +-
 .../calcite/rel/externalize/RelJsonReader.java     |   1 -
 .../org/apache/calcite/runtime/SqlFunctions.java   |  99 +++++++++-
 .../apache/calcite/sql/SqlJdbcFunctionCall.java    |   2 +
 .../calcite/sql/fun/SqlLibraryOperators.java       |  16 ++
 .../org/apache/calcite/util/BuiltInMethod.java     |  10 +
 .../calcite/sql/test/SqlOperatorBaseTest.java      | 150 +++++++--------
 .../java/org/apache/calcite/test/JdbcTest.java     | 201 ++++++++++++++-------
 site/_docs/reference.md                            |   6 +-
 14 files changed, 434 insertions(+), 166 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/DataContext.java b/core/src/main/java/org/apache/calcite/DataContext.java
index 994cce3..576c9e5 100644
--- a/core/src/main/java/org/apache/calcite/DataContext.java
+++ b/core/src/main/java/org/apache/calcite/DataContext.java
@@ -28,6 +28,7 @@ import com.google.common.base.CaseFormat;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.lang.reflect.Modifier;
+import java.util.Locale;
 import java.util.TimeZone;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -101,6 +102,12 @@ public interface DataContext {
     /** Writer to the standard output (stdout). */
     STDOUT("stdout", OutputStream.class),
 
+    /** Locale in which the current statement is executing.
+     * Affects the behavior of functions such as {@code DAYNAME} and
+     * {@code MONTHNAME}. Required; defaults to the root locale if the
+     * connection does not specify a locale. */
+    LOCALE("locale", Locale.class),
+
     /** Time zone in which the current statement is executing. Required;
      * defaults to the time zone of the JVM if the connection does not specify a
      * time zone. */
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 aaafe6a..db1da06 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
@@ -27,6 +27,7 @@ import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.ExpressionType;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.MemberExpression;
+import org.apache.calcite.linq4j.tree.MethodCallExpression;
 import org.apache.calcite.linq4j.tree.OptimizeShuttle;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
 import org.apache.calcite.linq4j.tree.Primitive;
@@ -91,6 +92,7 @@ import static org.apache.calcite.linq4j.tree.ExpressionType.NotEqual;
 import static org.apache.calcite.linq4j.tree.ExpressionType.OrElse;
 import static org.apache.calcite.linq4j.tree.ExpressionType.Subtract;
 import static org.apache.calcite.linq4j.tree.ExpressionType.UnaryPlus;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DAYNAME;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.DIFFERENCE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.FROM_BASE64;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_DEPTH;
@@ -101,6 +103,7 @@ import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_REMOVE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_STORAGE_SIZE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.JSON_TYPE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.LEFT;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.MONTHNAME;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.REPEAT;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.REVERSE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.RIGHT;
@@ -400,6 +403,14 @@ public class RexImpTable {
             BuiltInMethod.UNIX_DATE_CEIL.method), false);
 
     defineMethod(LAST_DAY, "lastDay", NullPolicy.STRICT);
+    defineImplementor(DAYNAME, NullPolicy.STRICT,
+        new PeriodNameImplementor("dayName",
+            BuiltInMethod.DAYNAME_WITH_TIMESTAMP,
+            BuiltInMethod.DAYNAME_WITH_DATE), false);
+    defineImplementor(MONTHNAME, NullPolicy.STRICT,
+        new PeriodNameImplementor("monthName",
+            BuiltInMethod.MONTHNAME_WITH_TIMESTAMP,
+            BuiltInMethod.MONTHNAME_WITH_DATE), false);
 
     map.put(IS_NULL, new IsXxxImplementor(null, false));
     map.put(IS_NOT_NULL, new IsXxxImplementor(null, true));
@@ -511,13 +522,13 @@ public class RexImpTable {
         JsonArrayAggImplementor
             .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD.method));
     defineImplementor(IS_JSON_VALUE, NullPolicy.NONE,
-            new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method), false);
+        new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method), false);
     defineImplementor(IS_JSON_OBJECT, NullPolicy.NONE,
-            new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method), false);
+        new MethodImplementor(BuiltInMethod.IS_JSON_OBJECT.method), false);
     defineImplementor(IS_JSON_ARRAY, NullPolicy.NONE,
-            new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method), false);
+        new MethodImplementor(BuiltInMethod.IS_JSON_ARRAY.method), false);
     defineImplementor(IS_JSON_SCALAR, NullPolicy.NONE,
-            new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method), false);
+        new MethodImplementor(BuiltInMethod.IS_JSON_SCALAR.method), false);
     defineImplementor(IS_NOT_JSON_VALUE, NullPolicy.NONE,
         NotImplementor.of(
             new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method)), false);
@@ -1596,8 +1607,8 @@ public class RexImpTable {
       add.currentBlock().add(
           Expressions.statement(
               Expressions.assign(acc.get(0),
-              Expressions.call(afi.isStatic ? null : acc.get(1), afi.addMethod,
-                  args))));
+                  Expressions.call(afi.isStatic ? null : acc.get(1), afi.addMethod,
+                      args))));
     }
 
     @Override protected Expression implementNotNullResult(AggContext info,
@@ -1637,9 +1648,9 @@ public class RexImpTable {
                       Expressions.subtract(add.currentPosition(),
                           Expressions.constant(1)),
                       add.currentPosition()),
-              Expressions.constant(0)),
-          Expressions.statement(
-              Expressions.assign(acc, computeNewRank(acc, add)))));
+                  Expressions.constant(0)),
+              Expressions.statement(
+                  Expressions.assign(acc, computeNewRank(acc, add)))));
       add.exitBlock();
       add.currentBlock().add(
           Expressions.ifThen(
@@ -2039,6 +2050,42 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code MONTHNAME} and {@code DAYNAME} functions.
+   * Each takes a {@link java.util.Locale} argument. */
+  private static class PeriodNameImplementor extends MethodNameImplementor {
+    private final BuiltInMethod timestampMethod;
+    private final BuiltInMethod dateMethod;
+
+    PeriodNameImplementor(String methodName, BuiltInMethod timestampMethod,
+        BuiltInMethod dateMethod) {
+      super(methodName);
+      this.timestampMethod = timestampMethod;
+      this.dateMethod = dateMethod;
+    }
+
+    @Override public Expression implement(RexToLixTranslator translator,
+        RexCall call, List<Expression> translatedOperands) {
+      Expression operand = translatedOperands.get(0);
+      final RelDataType type = call.operands.get(0).getType();
+      switch (type.getSqlTypeName()) {
+      case TIMESTAMP:
+        return getExpression(translator, operand, timestampMethod);
+      case DATE:
+        return getExpression(translator, operand, dateMethod);
+      default:
+        throw new AssertionError("unknown type " + type);
+      }
+    }
+
+    protected Expression getExpression(RexToLixTranslator translator,
+        Expression operand, BuiltInMethod builtInMethod) {
+      final MethodCallExpression locale =
+          Expressions.call(BuiltInMethod.LOCALE.method, translator.getRoot());
+      return Expressions.call(builtInMethod.method.getDeclaringClass(),
+          builtInMethod.method.getName(), operand, locale);
+    }
+  }
+
   /** Implementor for the {@code FLOOR} and {@code CEIL} functions. */
   private static class FloorImplementor extends MethodNameImplementor {
     final Method timestampMethod;
@@ -2061,8 +2108,10 @@ public class RexImpTable {
         case SMALLINT:
         case TINYINT:
           return translatedOperands.get(0);
+        default:
+          return super.implement(translator, call, translatedOperands);
         }
-        return super.implement(translator, call, translatedOperands);
+
       case 2:
         final Type type;
         final Method floorMethod;
@@ -2093,6 +2142,7 @@ public class RexImpTable {
         default:
           return call(operand, type, timeUnitRange.startUnit);
         }
+
       default:
         throw new AssertionError();
       }
@@ -2344,7 +2394,7 @@ public class RexImpTable {
       case MICROSECOND:
         operand = mod(operand, TimeUnit.MINUTE.multiplier.longValue());
         return Expressions.multiply(
-              operand, Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
+            operand, Expressions.constant(TimeUnit.SECOND.multiplier.longValue()));
       case EPOCH:
         switch (sqlTypeName) {
         case DATE:
@@ -2523,7 +2573,7 @@ public class RexImpTable {
         return translator.translate(arg, nullAs);
       }
       if (SqlTypeUtil.equalSansNullability(translator.typeFactory,
-              call.getType(), arg.getType())
+          call.getType(), arg.getType())
           && nullAs == NullAs.NULL
           && translator.deref(arg) instanceof RexLiteral) {
         return RexToLixTranslator.translateLiteral(
@@ -2612,10 +2662,10 @@ public class RexImpTable {
     }
   }
 
-    /** Implementor for SQL system functions.
-     *
-     * <p>Several of these are represented internally as constant values, set
-     * per execution. */
+  /** Implementor for SQL system functions.
+   *
+   * <p>Several of these are represented internally as constant values, set
+   * per execution. */
   private static class SystemFunctionImplementor
       implements CallImplementor {
     public Expression implement(
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
index 7952d7d..f977e7b 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfig.java
@@ -72,6 +72,8 @@ public interface CalciteConnectionConfig extends ConnectionConfig {
   SqlConformance conformance();
   /** @see CalciteConnectionProperty#TIME_ZONE */
   @Override String timeZone();
+  /** @see CalciteConnectionProperty#LOCALE */
+  String locale();
 }
 
 // End CalciteConnectionConfig.java
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
index 5b6397a..6ede95a 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -164,6 +164,11 @@ public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
     return CalciteConnectionProperty.TIME_ZONE.wrap(properties)
             .getString();
   }
+
+  public String locale() {
+    return CalciteConnectionProperty.LOCALE.wrap(properties)
+        .getString();
+  }
 }
 
 // End CalciteConnectionConfigImpl.java
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
index 3fe048e..d3b9cad 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -134,6 +134,13 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
    * Never null. */
   TIME_ZONE("timeZone", Type.STRING, TimeZone.getDefault().getID(), false),
 
+  /** Returns the locale from the connect string.
+   * If the locale is not set, returns the root locale.
+   * Never null.
+   * Examples of valid locales: 'en', 'en_US',
+   * 'de_DE', '_GB', 'en_US_WIN', 'de__POSIX', 'fr__MAC', ''. */
+  LOCALE("locale", Type.STRING, Locale.ROOT.toString(), false),
+
   /** If the planner should try de-correlating as much as it is possible.
    * If true (the default), Calcite de-correlates the plan. */
   FORCE_DECORRELATE("forceDecorrelate", Type.BOOLEAN, true, false),
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index 8746f53..232f733 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -63,6 +63,7 @@ import org.apache.calcite.sql.validate.SqlValidatorWithHints;
 import org.apache.calcite.tools.RelRunner;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Holder;
+import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -76,6 +77,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Properties;
@@ -414,6 +416,9 @@ abstract class CalciteConnectionImpl
       final long currentOffset = localOffset;
       final String user = "sa";
       final String systemUser = System.getProperty("user.name");
+      final String localeName = connection.config().locale();
+      final Locale locale = localeName != null
+          ? Util.parseLocale(localeName) : Locale.ROOT;
 
       // Give a hook chance to alter standard input, output, error streams.
       final Holder<Object[]> streamHolder =
@@ -427,6 +432,7 @@ abstract class CalciteConnectionImpl
           .put(Variable.TIME_ZONE.camelName, timeZone)
           .put(Variable.USER.camelName, user)
           .put(Variable.SYSTEM_USER.camelName, systemUser)
+          .put(Variable.LOCALE.camelName, locale)
           .put(Variable.STDIN.camelName, streamHolder.get()[0])
           .put(Variable.STDOUT.camelName, streamHolder.get()[1])
           .put(Variable.STDERR.camelName, streamHolder.get()[2]);
@@ -465,9 +471,9 @@ abstract class CalciteConnectionImpl
               : ImmutableList.of(schemaName);
       final SqlValidatorWithHints validator =
           new SqlAdvisorValidator(SqlStdOperatorTable.instance(),
-          new CalciteCatalogReader(rootSchema,
-              schemaPath, typeFactory, con.config()),
-          typeFactory, SqlConformanceEnum.DEFAULT);
+              new CalciteCatalogReader(rootSchema,
+                  schemaPath, typeFactory, con.config()),
+              typeFactory, SqlConformanceEnum.DEFAULT);
       final CalciteConnectionConfig config = con.config();
       // This duplicates org.apache.calcite.prepare.CalcitePrepareImpl.prepare2_
       final SqlParser.Config parserConfig = SqlParser.configBuilder()
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
index 0403cc5..b7d7e8a 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
@@ -83,7 +83,6 @@ public class RelJsonReader {
     @SuppressWarnings("unchecked")
     final List<Map<String, Object>> rels = (List) o.get("rels");
     readRels(rels);
-    System.out.println(lastRel);
     return lastRel;
   }
 
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 fb9e32c..a4fa957 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -52,6 +52,8 @@ import java.math.RoundingMode;
 import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
+import java.time.LocalDate;
+import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Base64;
@@ -94,6 +96,12 @@ public class SqlFunctions {
 
   private static final TimeZone LOCAL_TZ = TimeZone.getDefault();
 
+  private static final DateTimeFormatter ROOT_DAY_FORMAT =
+      DateTimeFormatter.ofPattern("EEEE", Locale.ROOT);
+
+  private static final DateTimeFormatter ROOT_MONTH_FORMAT =
+      DateTimeFormatter.ofPattern("MMMM", Locale.ROOT);
+
   private static final Soundex SOUNDEX = new Soundex();
 
   private static final int SOUNDEX_LENGTH = 4;
@@ -466,16 +474,16 @@ public class SqlFunctions {
   /** SQL {@code OVERLAY} function applied to binary strings. */
   public static ByteString overlay(ByteString s, ByteString r, int start) {
     return s.substring(0, start - 1)
-           .concat(r)
-           .concat(s.substring(start - 1 + r.length()));
+        .concat(r)
+        .concat(s.substring(start - 1 + r.length()));
   }
 
   /** SQL {@code OVERLAY} function applied to binary strings. */
   public static ByteString overlay(ByteString s, ByteString r, int start,
       int length) {
     return s.substring(0, start - 1)
-           .concat(r)
-           .concat(s.substring(start - 1 + length));
+        .concat(r)
+        .concat(s.substring(start - 1 + length));
   }
 
   /** SQL {@code LIKE} function. */
@@ -1755,8 +1763,8 @@ public class SqlFunctions {
   public static float toFloat(Object o) {
     return o instanceof Float ? (Float) o
         : o instanceof Number ? toFloat((Number) o)
-            : o instanceof String ? toFloat((String) o)
-                : (Float) cannotConvert(o, float.class);
+        : o instanceof String ? toFloat((String) o)
+        : (Float) cannotConvert(o, float.class);
   }
 
   public static double toDouble(String s) {
@@ -2062,6 +2070,78 @@ public class SqlFunctions {
     return DateTimeUtils.ymdToUnixDate(y0, m0, last);
   }
 
+  /**
+   * SQL {@code DAYNAME} function, applied to a TIMESTAMP argument.
+   *
+   * @param timestamp Milliseconds from epoch
+   * @param locale Locale
+   * @return Name of the weekday in the given locale
+   */
+  public static String dayNameWithTimestamp(long timestamp, Locale locale) {
+    return timeStampToLocalDate(timestamp)
+        .format(ROOT_DAY_FORMAT.withLocale(locale));
+  }
+
+  /**
+   * SQL {@code DAYNAME} function, applied to a DATE argument.
+   *
+   * @param date Days since epoch
+   * @param locale Locale
+   * @return Name of the weekday in the given locale
+   */
+  public static String dayNameWithDate(int date, Locale locale) {
+    return dateToLocalDate(date)
+        .format(ROOT_DAY_FORMAT.withLocale(locale));
+  }
+
+  /**
+   * SQL {@code MONTHNAME} function, applied to a TIMESTAMP argument.
+   *
+   * @param timestamp Milliseconds from epoch
+   * @param locale Locale
+   * @return Name of the month in the given locale
+   */
+  public static String monthNameWithTimestamp(long timestamp, Locale locale) {
+    return timeStampToLocalDate(timestamp)
+        .format(ROOT_MONTH_FORMAT.withLocale(locale));
+  }
+
+  /**
+   * SQL {@code MONTHNAME} function, applied to a DATE argument.
+   *
+   * @param date Days from epoch
+   * @param locale Locale
+   * @return Name of the month in the given locale
+   */
+  public static String monthNameWithDate(int date, Locale locale) {
+    return dateToLocalDate(date)
+        .format(ROOT_MONTH_FORMAT.withLocale(locale));
+  }
+
+  /**
+   * Converts a date (days since epoch) to a {@link LocalDate}.
+   *
+   * @param date days since epoch
+   * @return localDate
+   */
+  private static LocalDate dateToLocalDate(int date) {
+    int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
+    int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
+    int d0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.DAY, date);
+    return LocalDate.of(y0, m0, d0);
+  }
+
+  /**
+   * Converts a timestamp (milliseconds since epoch) to a {@link LocalDate}.
+   *
+   * @param timestamp milliseconds from epoch
+   * @return localDate
+   */
+  private static LocalDate timeStampToLocalDate(long timestamp) {
+    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
+    return dateToLocalDate(date);
+  }
+
   /** SQL {@code CURRENT_TIMESTAMP} function. */
   @NonDeterministic
   public static long currentTimestamp(DataContext root) {
@@ -2121,6 +2201,11 @@ public class SqlFunctions {
     return Objects.requireNonNull(DataContext.Variable.SYSTEM_USER.get(root));
   }
 
+  @NonDeterministic
+  public static Locale locale(DataContext root) {
+    return (Locale) DataContext.Variable.LOCALE.get(root);
+  }
+
   /** SQL {@code TRANSLATE(string, search_chars, replacement_chars)}
    * function. */
   public static String translate3(String s, String search, String replacement) {
@@ -2369,7 +2454,7 @@ public class SqlFunctions {
         return (Function1) LIST_AS_ENUMERABLE;
       } else {
         return row -> p2(new Object[] { row }, fieldCounts, withOrdinality,
-              inputTypes);
+            inputTypes);
       }
     }
     return lists -> p2((Object[]) lists, fieldCounts, withOrdinality,
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
index 7a8f3b3..d778908 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJdbcFunctionCall.java
@@ -723,8 +723,10 @@ public class SqlJdbcFunctionCall extends SqlFunction {
       map.put("DAYOFYEAR", simple(SqlStdOperatorTable.DAYOFYEAR));
       map.put("DAYOFMONTH", simple(SqlStdOperatorTable.DAYOFMONTH));
       map.put("DAYOFWEEK", simple(SqlStdOperatorTable.DAYOFWEEK));
+      map.put("DAYNAME", simple(SqlLibraryOperators.DAYNAME));
       map.put("HOUR", simple(SqlStdOperatorTable.HOUR));
       map.put("MINUTE", simple(SqlStdOperatorTable.MINUTE));
+      map.put("MONTHNAME", simple(SqlLibraryOperators.MONTHNAME));
       map.put("SECOND", simple(SqlStdOperatorTable.SECOND));
 
       map.put("CURDATE", simple(SqlStdOperatorTable.CURRENT_DATE));
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index 94afbbe..15a7afc 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -154,6 +154,22 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {MYSQL})
   public static final SqlFunction JSON_STORAGE_SIZE = new SqlJsonStorageSizeFunction();
 
+  /** The "MONTHNAME(datetime)" function; returns the name of the month,
+   * in the current locale, of a TIMESTAMP or DATE argument. */
+  @LibraryOperator(libraries = {MYSQL})
+  public static final SqlFunction MONTHNAME =
+      new SqlFunction("MONTHNAME", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.VARCHAR_2000, null, OperandTypes.DATETIME,
+          SqlFunctionCategory.TIMEDATE);
+
+  /** The "DAYNAME(datetime)" function; returns the name of the day of the week,
+   * in the current locale, of a TIMESTAMP or DATE argument. */
+  @LibraryOperator(libraries = {MYSQL})
+  public static final SqlFunction DAYNAME =
+      new SqlFunction("DAYNAME", SqlKind.OTHER_FUNCTION,
+          ReturnTypes.VARCHAR_2000, null, OperandTypes.DATETIME,
+          SqlFunctionCategory.TIMEDATE);
+
   @LibraryOperator(libraries = {MYSQL, POSTGRESQL})
   public static final SqlFunction LEFT =
       new SqlFunction("LEFT", SqlKind.OTHER_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 30d91cd..6b4d35e 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -114,6 +114,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.TimeZone;
@@ -413,6 +414,14 @@ public enum BuiltInMethod {
   UNIX_TIMESTAMP_CEIL(DateTimeUtils.class, "unixTimestampCeil",
       TimeUnitRange.class, long.class),
   LAST_DAY(SqlFunctions.class, "lastDay", int.class),
+  DAYNAME_WITH_TIMESTAMP(SqlFunctions.class,
+      "dayNameWithTimestamp", long.class, Locale.class),
+  DAYNAME_WITH_DATE(SqlFunctions.class,
+      "dayNameWithDate", int.class, Locale.class),
+  MONTHNAME_WITH_TIMESTAMP(SqlFunctions.class,
+      "monthNameWithTimestamp", long.class, Locale.class),
+  MONTHNAME_WITH_DATE(SqlFunctions.class,
+      "monthNameWithDate", int.class, Locale.class),
   CURRENT_TIMESTAMP(SqlFunctions.class, "currentTimestamp", DataContext.class),
   CURRENT_TIME(SqlFunctions.class, "currentTime", DataContext.class),
   CURRENT_DATE(SqlFunctions.class, "currentDate", DataContext.class),
@@ -421,6 +430,7 @@ public enum BuiltInMethod {
   TIME_ZONE(SqlFunctions.class, "timeZone", DataContext.class),
   USER(SqlFunctions.class, "user", DataContext.class),
   SYSTEM_USER(SqlFunctions.class, "systemUser", DataContext.class),
+  LOCALE(SqlFunctions.class, "locale", DataContext.class),
   BOOLEAN_TO_STRING(SqlFunctions.class, "toString", boolean.class),
   JDBC_ARRAY_TO_LIST(SqlFunctions.class, "arrayToList", java.sql.Array.class),
   OBJECT_TO_STRING(Object.class, "toString"),
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index eee2dc0..544e8ec 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -1927,9 +1927,10 @@ public abstract class SqlOperatorBaseTest {
     // Time and Date Functions
     tester.checkType("{fn CURDATE()}", "DATE NOT NULL");
     tester.checkType("{fn CURTIME()}", "TIME(0) NOT NULL");
-    if (false) {
-      tester.checkScalar("{fn DAYNAME(date)}", null, "");
-    }
+    tester.checkScalar("{fn DAYNAME(DATE '2014-12-10')}",
+        // Day names in root locale changed from long to short in JDK 9
+        TestUtil.getJavaMajorVersion() <= 8 ? "Wednesday" : "Wed",
+        "VARCHAR(2000) NOT NULL");
     tester.checkScalar("{fn DAYOFMONTH(DATE '2014-12-10')}", 10,
         "BIGINT NOT NULL");
     if (Bug.CALCITE_2539_FIXED) {
@@ -1945,16 +1946,17 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar("{fn MINUTE(TIMESTAMP '2014-12-10 12:34:56')}", 34,
         "BIGINT NOT NULL");
     tester.checkScalar("{fn MONTH(DATE '2014-12-10')}", 12, "BIGINT NOT NULL");
-    if (false) {
-      tester.checkScalar("{fn MONTHNAME(date)}", null, "");
-    }
+    tester.checkScalar("{fn MONTHNAME(DATE '2014-12-10')}",
+        // Month names in root locale changed from long to short in JDK 9
+        TestUtil.getJavaMajorVersion() <= 8 ? "December" : "Dec",
+        "VARCHAR(2000) NOT NULL");
     tester.checkType("{fn NOW()}", "TIMESTAMP(0) NOT NULL");
     tester.checkScalar("{fn QUARTER(DATE '2014-12-10')}", "4",
         "BIGINT NOT NULL");
     tester.checkScalar("{fn SECOND(TIMESTAMP '2014-12-10 12:34:56')}", 56,
         "BIGINT NOT NULL");
     tester.checkScalar("{fn TIMESTAMPADD(HOUR, 5,"
-        + " TIMESTAMP '2014-03-29 12:34:56')}",
+            + " TIMESTAMP '2014-03-29 12:34:56')}",
         "2014-03-29 17:34:56", "TIMESTAMP(0) NOT NULL");
     tester.checkScalar("{fn TIMESTAMPDIFF(HOUR,"
         + " TIMESTAMP '2014-03-29 12:34:56',"
@@ -2190,7 +2192,7 @@ public abstract class SqlOperatorBaseTest {
     // submit as non-runtime because the janino exception does not have
     // error position information and the framework is unhappy with that.
     tester1.checkFails(
-            "3 % case 'a' when 'a' then 0 end", DIVISION_BY_ZERO_MESSAGE, true);
+        "3 % case 'a' when 'a' then 0 end", DIVISION_BY_ZERO_MESSAGE, true);
   }
 
   @Test public void testDivideOperator() {
@@ -4528,7 +4530,7 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails("json_value('{\"foo\":\"100\"}', 'strict $.foo' returning boolean)",
         INVALID_CHAR_MESSAGE, true);
     tester.checkScalar("json_value('{\"foo\":100}', 'lax $.foo1' returning integer "
-            + "null on empty)", null, "INTEGER");
+        + "null on empty)", null, "INTEGER");
     tester.checkScalar("json_value('{\"foo\":\"100\"}', 'strict $.foo1' returning boolean "
         + "null on error)", null, "BOOLEAN");
 
@@ -4729,24 +4731,24 @@ public abstract class SqlOperatorBaseTest {
   @Test public void testJsonType() {
     tester.setFor(SqlLibraryOperators.JSON_TYPE);
     tester.checkString("json_type('\"1\"')",
-            "STRING", "VARCHAR(20)");
+        "STRING", "VARCHAR(20)");
     tester.checkString("json_type('1')",
-            "INTEGER", "VARCHAR(20)");
+        "INTEGER", "VARCHAR(20)");
     tester.checkString("json_type('11.45')",
-            "DOUBLE", "VARCHAR(20)");
+        "DOUBLE", "VARCHAR(20)");
     tester.checkString("json_type('true')",
-            "BOOLEAN", "VARCHAR(20)");
+        "BOOLEAN", "VARCHAR(20)");
     tester.checkString("json_type('null')",
-            "NULL", "VARCHAR(20)");
+        "NULL", "VARCHAR(20)");
     tester.checkNull("json_type(cast(null as varchar(1)))");
     tester.checkString("json_type('{\"a\": [10, true]}')",
-            "OBJECT", "VARCHAR(20)");
+        "OBJECT", "VARCHAR(20)");
     tester.checkString("json_type('{}')",
-            "OBJECT", "VARCHAR(20)");
+        "OBJECT", "VARCHAR(20)");
     tester.checkString("json_type('[10, true]')",
-            "ARRAY", "VARCHAR(20)");
+        "ARRAY", "VARCHAR(20)");
     tester.checkString("json_type('\"2019-01-27 21:24:00\"')",
-            "STRING", "VARCHAR(20)");
+        "STRING", "VARCHAR(20)");
 
     // nulls
     tester.checkFails("json_type(^null^)",
@@ -4757,29 +4759,29 @@ public abstract class SqlOperatorBaseTest {
   @Test public void testJsonDepth() {
     tester.setFor(SqlLibraryOperators.JSON_DEPTH);
     tester.checkString("json_depth('1')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('11.45')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('true')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('\"2019-01-27 21:24:00\"')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('{}')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('[]')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_depth('null')",
-            null, "INTEGER");
+        null, "INTEGER");
     tester.checkString("json_depth(cast(null as varchar(1)))",
-            null, "INTEGER");
+        null, "INTEGER");
     tester.checkString("json_depth('[10, true]')",
-            "2", "INTEGER");
+        "2", "INTEGER");
     tester.checkString("json_depth('[[], {}]')",
-            "2", "INTEGER");
+        "2", "INTEGER");
     tester.checkString("json_depth('{\"a\": [10, true]}')",
-            "3", "INTEGER");
+        "3", "INTEGER");
     tester.checkString("json_depth('[10, {\"a\": [[1,2]]}]')",
-            "5", "INTEGER");
+        "5", "INTEGER");
 
     // nulls
     tester.checkFails("json_depth(^null^)",
@@ -4790,51 +4792,51 @@ public abstract class SqlOperatorBaseTest {
   @Test public void testJsonLength() {
     // no path context
     tester.checkString("json_length('{}')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('[]')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('{\"foo\":100}')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_length('{\"a\": 1, \"b\": {\"c\": 30}}')",
-            "2", "INTEGER");
+        "2", "INTEGER");
     tester.checkString("json_length('[1, 2, {\"a\": 3}]')",
-            "3", "INTEGER");
+        "3", "INTEGER");
 
     // lax test
     tester.checkString("json_length('{}', 'lax $')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('[]', 'lax $')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('{\"foo\":100}', 'lax $')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_length('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $')",
-            "2", "INTEGER");
+        "2", "INTEGER");
     tester.checkString("json_length('[1, 2, {\"a\": 3}]', 'lax $')",
-            "3", "INTEGER");
+        "3", "INTEGER");
     tester.checkString("json_length('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $.b')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_length('{\"foo\":100}', 'lax $.foo1')",
-            null, "INTEGER");
+        null, "INTEGER");
 
     // strict test
     tester.checkString("json_length('{}', 'strict $')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('[]', 'strict $')",
-            "0", "INTEGER");
+        "0", "INTEGER");
     tester.checkString("json_length('{\"foo\":100}', 'strict $')",
-            "1", "INTEGER");
+        "1", "INTEGER");
     tester.checkString("json_length('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $')",
-            "2", "INTEGER");
+        "2", "INTEGER");
     tester.checkString("json_length('[1, 2, {\"a\": 3}]', 'strict $')",
-            "3", "INTEGER");
+        "3", "INTEGER");
     tester.checkString("json_length('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $.b')",
-            "1", "INTEGER");
+        "1", "INTEGER");
 
     // catch error test
     tester.checkFails("json_length('{\"foo\":100}', 'invalid $.foo')",
-            "(?s).*Illegal jsonpath spec.*", true);
+        "(?s).*Illegal jsonpath spec.*", true);
     tester.checkFails("json_length('{\"foo\":100}', 'strict $.foo1')",
-            "(?s).*No results for path.*", true);
+        "(?s).*No results for path.*", true);
 
     // nulls
     tester.checkFails("json_length(^null^)",
@@ -4845,51 +4847,51 @@ public abstract class SqlOperatorBaseTest {
   @Test public void testJsonKeys() {
     // no path context
     tester.checkString("json_keys('{}')",
-            "[]", "VARCHAR(2000)");
+        "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}')",
-            "[\"foo\"]", "VARCHAR(2000)");
+        "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}')",
-            "[\"a\",\"b\"]", "VARCHAR(2000)");
+        "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
 
     // lax test
     tester.checkString("json_keys('{}', 'lax $')",
-            "[]", "VARCHAR(2000)");
+        "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]', 'lax $')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'lax $')",
-            "[\"foo\"]", "VARCHAR(2000)");
+        "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $')",
-            "[\"a\",\"b\"]", "VARCHAR(2000)");
+        "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]', 'lax $')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'lax $.b')",
-            "[\"c\"]", "VARCHAR(2000)");
+        "[\"c\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'lax $.foo1')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
 
     // strict test
     tester.checkString("json_keys('{}', 'strict $')",
-            "[]", "VARCHAR(2000)");
+        "[]", "VARCHAR(2000)");
     tester.checkString("json_keys('[]', 'strict $')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"foo\":100}', 'strict $')",
-            "[\"foo\"]", "VARCHAR(2000)");
+        "[\"foo\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $')",
-            "[\"a\",\"b\"]", "VARCHAR(2000)");
+        "[\"a\",\"b\"]", "VARCHAR(2000)");
     tester.checkString("json_keys('[1, 2, {\"a\": 3}]', 'strict $')",
-            "null", "VARCHAR(2000)");
+        "null", "VARCHAR(2000)");
     tester.checkString("json_keys('{\"a\": 1, \"b\": {\"c\": 30}}', 'strict $.b')",
-            "[\"c\"]", "VARCHAR(2000)");
+        "[\"c\"]", "VARCHAR(2000)");
 
     // catch error test
     tester.checkFails("json_keys('{\"foo\":100}', 'invalid $.foo')",
-            "(?s).*Illegal jsonpath spec.*", true);
+        "(?s).*Illegal jsonpath spec.*", true);
     tester.checkFails("json_keys('{\"foo\":100}', 'strict $.foo1')",
-            "(?s).*No results for path.*", true);
+        "(?s).*No results for path.*", true);
 
     // nulls
     tester.checkFails("json_keys(^null^)",
@@ -6379,11 +6381,11 @@ public abstract class SqlOperatorBaseTest {
     tester.checkBoolean("(multiset['a', 'b', 'c'] "
             + "multiset union distinct multiset['c', 'd', 'e'])"
             + " submultiset of multiset['a', 'b', 'c', 'd', 'e']",
-         Boolean.TRUE);
+        Boolean.TRUE);
     tester.checkBoolean("(multiset['a', 'b', 'c'] "
             + "multiset union distinct multiset['c', 'd', 'e'])"
             + " submultiset of multiset['a', 'b', 'c', 'd', 'e']",
-         Boolean.TRUE);
+        Boolean.TRUE);
     tester.checkScalar(
         "multiset[cast(null as double)] multiset union multiset[cast(null as double)]",
         "[null, null]",
@@ -6905,7 +6907,7 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar(
         "extract(epoch from date '2008-2-23')",
         "1203724800", // number of seconds elapsed since timestamp
-                      // '1970-01-01 00:00:00' for given date
+        // '1970-01-01 00:00:00' for given date
         "BIGINT NOT NULL");
 
     tester.checkScalar(
@@ -7033,7 +7035,7 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar(
         "extract(epoch from timestamp '2008-2-23 12:34:56')",
         "1203770096", // number of seconds elapsed since timestamp
-                      // '1970-01-01 00:00:00' for given date
+        // '1970-01-01 00:00:00' for given date
         "BIGINT NOT NULL");
 
     tester.checkScalar(
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 cc15aa4..9f62540 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -137,6 +137,7 @@ import java.util.Set;
 import java.util.TimeZone;
 import java.util.function.Consumer;
 import java.util.regex.Pattern;
+import java.util.stream.Stream;
 import javax.sql.DataSource;
 
 import static org.apache.calcite.test.Matchers.isLinux;
@@ -1131,7 +1132,7 @@ public class JdbcTest {
       "EXPR$0=86805\n",
       "select \"time_by_day\".\"the_year\" as \"c0\" from \"time_by_day\" as \"time_by_day\" group by \"time_by_day\".\"the_year\" order by \"time_by_day\".\"the_year\" ASC",
       "c0=1997\n"
-        + "c0=1998\n",
+          + "c0=1998\n",
       "select \"store\".\"store_country\" as \"c0\" from \"store\" as \"store\" where UPPER(\"store\".\"store_country\") = UPPER('USA') group by \"store\".\"store_country\" order by \"store\".\"store_country\" ASC",
       "c0=USA\n",
       "select \"store\".\"store_state\" as \"c0\" from \"store\" as \"store\" where (\"store\".\"store_country\" = 'USA') and UPPER(\"store\".\"store_state\") = UPPER('CA') group by \"store\".\"store_state\" order by \"store\".\"store_state\" ASC",
@@ -2702,17 +2703,17 @@ public class JdbcTest {
         .with(CalciteAssert.Config.FOODMART_CLONE)
         .query(s)
         .returns("the_month=April; c=30; c2=10\n"
-                + "the_month=August; c=31; c2=11\n"
-                + "the_month=December; c=31; c2=11\n"
-                + "the_month=February; c=28; c2=8\n"
-                + "the_month=January; c=31; c2=11\n"
-                + "the_month=July; c=31; c2=11\n"
-                + "the_month=June; c=30; c2=10\n"
-                + "the_month=March; c=31; c2=11\n"
-                + "the_month=May; c=31; c2=11\n"
-                + "the_month=November; c=30; c2=10\n"
-                + "the_month=October; c=31; c2=11\n"
-                + "the_month=September; c=30; c2=10\n");
+            + "the_month=August; c=31; c2=11\n"
+            + "the_month=December; c=31; c2=11\n"
+            + "the_month=February; c=28; c2=8\n"
+            + "the_month=January; c=31; c2=11\n"
+            + "the_month=July; c=31; c2=11\n"
+            + "the_month=June; c=30; c2=10\n"
+            + "the_month=March; c=31; c2=11\n"
+            + "the_month=May; c=31; c2=11\n"
+            + "the_month=November; c=30; c2=10\n"
+            + "the_month=October; c=31; c2=11\n"
+            + "the_month=September; c=30; c2=10\n");
   }
 
   /** Tests a simple IN query implemented as a semi-join. */
@@ -3495,14 +3496,14 @@ public class JdbcTest {
             + "        MINa2w0,\n"
             + "        COUNTa3w0});"
             : "_list.add(new Object[] {\n"
-                + "        row[0],\n" // box-unbox is optimized
-                + "        row[1],\n"
-                + "        row[2],\n"
-                + "        row[3],\n"
-                + "        a0w0,\n"
-                + "        a1w0,\n"
-                + "        a2w0,\n"
-                + "        a3w0});")
+            + "        row[0],\n" // box-unbox is optimized
+            + "        row[1],\n"
+            + "        row[2],\n"
+            + "        row[3],\n"
+            + "        a0w0,\n"
+            + "        a1w0,\n"
+            + "        a2w0,\n"
+            + "        a3w0});")
         .planContains("return new Object[] {\n"
             + "                  current[1],\n"
             + "                  current[0],\n"
@@ -5340,7 +5341,7 @@ public class JdbcTest {
 
         // all table types
         try (ResultSet r =
-             metaData.getTables(null, "adhoc", null, null)) {
+                 metaData.getTables(null, "adhoc", null, null)) {
           assertEquals(
               "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n"
                   + "TABLE_CAT=null; TABLE_SCHEM=adhoc; TABLE_NAME=MUTABLE_EMPLOYEES; TABLE_TYPE=TABLE; REMARKS=null; TYPE_CAT=null; TYPE_SCHEM=null; TYPE_NAME=null; SELF_REFERENCING_COL_NAME=null; REF_GENERATION=null\n"
@@ -5797,19 +5798,19 @@ public class JdbcTest {
       // timetz: 15:00:00+03
       ts = rs.getTimestamp(c);
       assertEquals(43200000L, ts.getTime());    // 1970-01-01 15:00:00 +0300 ->
-                                                // 1970-01-01 13:00:00 +0100
+      // 1970-01-01 13:00:00 +0100
       ts = rs.getTimestamp(c, cUtc);
       assertEquals(43200000L, ts.getTime());    // 1970-01-01 15:00:00 +0300 ->
-                                                // 1970-01-01 12:00:00 +0000
+      // 1970-01-01 12:00:00 +0000
       ts = rs.getTimestamp(c, cGmt03);
       assertEquals(43200000L, ts.getTime());    // 1970-01-01 15:00:00 +0300 ->
-                                                // 1970-01-01 15:00:00 +0300
+      // 1970-01-01 15:00:00 +0300
       ts = rs.getTimestamp(c, cGmt05);
       assertEquals(43200000L, ts.getTime());    // 1970-01-01 15:00:00 +0300 ->
-                                                // 1970-01-01 07:00:00 -0500
+      // 1970-01-01 07:00:00 -0500
       ts = rs.getTimestamp(c, cGmt13);
       assertEquals(43200000L, ts.getTime());    // 1970-01-01 15:00:00 +0300 ->
-                                                // 1970-01-02 01:00:00 +1300
+      // 1970-01-02 01:00:00 +1300
       ++c;
     }
 
@@ -5831,6 +5832,48 @@ public class JdbcTest {
     assertTrue(!rs.next());
   }
 
+  /** Test for MONTHNAME and DAYNAME functions in two locales. */
+  @Test public void testMonthName() {
+    final String sql = "SELECT * FROM (VALUES(\n"
+        + " monthname(TIMESTAMP '1969-01-01 00:00:00'),\n"
+        + " monthname(DATE '1969-01-01'),\n"
+        + " monthname(DATE '2019-02-10'),\n"
+        + " monthname(TIMESTAMP '2019-02-10 02:10:12'),\n"
+        + " dayname(TIMESTAMP '1969-01-01 00:00:00'),\n"
+        + " dayname(DATE '1969-01-01'),\n"
+        + " dayname(DATE '2019-02-10'),\n"
+        + " dayname(TIMESTAMP '2019-02-10 02:10:12')\n"
+        + ")) AS t(t0, t1, t2, t3, t4, t5, t6, t7)";
+    Stream.of(TestLocale.values()).forEach(t -> {
+      try {
+        CalciteAssert.that()
+            .with(CalciteConnectionProperty.LOCALE, t.localeName)
+            .with(CalciteConnectionProperty.FUN, "mysql")
+            .doWithConnection(connection -> {
+              try (Statement statement = connection.createStatement()) {
+                try (ResultSet rs = statement.executeQuery(sql)) {
+                  assertThat(rs.next(), is(true));
+                  assertThat(rs.getString(1), is(t.january));
+                  assertThat(rs.getString(2), is(t.january));
+                  assertThat(rs.getString(3), is(t.february));
+                  assertThat(rs.getString(4), is(t.february));
+                  assertThat(rs.getString(5), is(t.wednesday));
+                  assertThat(rs.getString(6), is(t.wednesday));
+                  assertThat(rs.getString(7), is(t.sunday));
+                  assertThat(rs.getString(8), is(t.sunday));
+                  assertThat(rs.next(), is(false));
+                }
+              } catch (SQLException e) {
+                throw TestUtil.rethrow(e);
+              }
+            });
+      } catch (Exception e) {
+        System.out.println(t.localeName + ":" + Locale.getDefault().toString());
+        throw TestUtil.rethrow(e);
+      }
+    });
+  }
+
   /** Tests accessing a column in a JDBC source whose type is DATE. */
   @Test
   public void testGetDate() throws Exception {
@@ -6562,20 +6605,20 @@ public class JdbcTest {
     Properties info = new Properties();
     info.put("model",
         "inline:"
-        + "{\n"
-        + "  version: '1.0',\n"
-        + "  defaultSchema: 'BASEJDBC',\n"
-        + "  schemas: [\n"
-        + "     {\n"
-        + "       type: 'jdbc',\n"
-        + "       name: 'BASEJDBC',\n"
-        + "       jdbcDriver: '" + jdbcDriver.class.getName() + "',\n"
-        + "       jdbcUrl: '" + hsqldbMemUrl + "',\n"
-        + "       jdbcCatalog: null,\n"
-        + "       jdbcSchema: null\n"
-        + "     }\n"
-        + "  ]\n"
-        + "}");
+            + "{\n"
+            + "  version: '1.0',\n"
+            + "  defaultSchema: 'BASEJDBC',\n"
+            + "  schemas: [\n"
+            + "     {\n"
+            + "       type: 'jdbc',\n"
+            + "       name: 'BASEJDBC',\n"
+            + "       jdbcDriver: '" + jdbcDriver.class.getName() + "',\n"
+            + "       jdbcUrl: '" + hsqldbMemUrl + "',\n"
+            + "       jdbcCatalog: null,\n"
+            + "       jdbcSchema: null\n"
+            + "     }\n"
+            + "  ]\n"
+            + "}");
 
     Connection calciteConnection =
         DriverManager.getConnection("jdbc:calcite:", info);
@@ -6675,20 +6718,20 @@ public class JdbcTest {
    * ClassCastException in table from CloneSchema</a>. */
   @Test public void testNullableNumericColumnInCloneSchema() {
     CalciteAssert.model("{\n"
-            + "  version: '1.0',\n"
-            + "  defaultSchema: 'SCOTT_CLONE',\n"
-            + "  schemas: [ {\n"
-            + "    name: 'SCOTT_CLONE',\n"
-            + "    type: 'custom',\n"
-            + "    factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',\n"
-            + "    operand: {\n"
-            + "      jdbcDriver: '" + JdbcTest.SCOTT.driver + "',\n"
-            + "      jdbcUser: '" + JdbcTest.SCOTT.username + "',\n"
-            + "      jdbcPassword: '" + JdbcTest.SCOTT.password + "',\n"
-            + "      jdbcUrl: '" + JdbcTest.SCOTT.url + "',\n"
-            + "      jdbcSchema: 'SCOTT'\n"
-            + "   } } ]\n"
-            + "}")
+        + "  version: '1.0',\n"
+        + "  defaultSchema: 'SCOTT_CLONE',\n"
+        + "  schemas: [ {\n"
+        + "    name: 'SCOTT_CLONE',\n"
+        + "    type: 'custom',\n"
+        + "    factory: 'org.apache.calcite.adapter.clone.CloneSchema$Factory',\n"
+        + "    operand: {\n"
+        + "      jdbcDriver: '" + JdbcTest.SCOTT.driver + "',\n"
+        + "      jdbcUser: '" + JdbcTest.SCOTT.username + "',\n"
+        + "      jdbcPassword: '" + JdbcTest.SCOTT.password + "',\n"
+        + "      jdbcUrl: '" + JdbcTest.SCOTT.url + "',\n"
+        + "      jdbcSchema: 'SCOTT'\n"
+        + "   } } ]\n"
+        + "}")
         .query("select * from emp")
         .returns(input -> {
           final StringBuilder buf = new StringBuilder();
@@ -6845,13 +6888,13 @@ public class JdbcTest {
 
   @Test public void testJsonDepth() {
     CalciteAssert.that()
-            .query("SELECT JSON_DEPTH(v) AS c1\n"
-                    + ",JSON_DEPTH(JSON_VALUE(v, 'lax $.b' ERROR ON ERROR)) AS c2\n"
-                    + ",JSON_DEPTH(JSON_VALUE(v, 'strict $.a[0]' ERROR ON ERROR)) AS c3\n"
-                    + ",JSON_DEPTH(JSON_VALUE(v, 'strict $.a[1]' ERROR ON ERROR)) AS c4\n"
-                    + "FROM (VALUES ('{\"a\": [10, true],\"b\": \"[10, true]\"}')) AS t(v)\n"
-                    + "limit 10")
-            .returns("C1=3; C2=2; C3=1; C4=1\n");
+        .query("SELECT JSON_DEPTH(v) AS c1\n"
+            + ",JSON_DEPTH(JSON_VALUE(v, 'lax $.b' ERROR ON ERROR)) AS c2\n"
+            + ",JSON_DEPTH(JSON_VALUE(v, 'strict $.a[0]' ERROR ON ERROR)) AS c3\n"
+            + ",JSON_DEPTH(JSON_VALUE(v, 'strict $.a[1]' ERROR ON ERROR)) AS c4\n"
+            + "FROM (VALUES ('{\"a\": [10, true],\"b\": \"[10, true]\"}')) AS t(v)\n"
+            + "limit 10")
+        .returns("C1=3; C2=2; C3=1; C4=1\n");
   }
 
   @Test public void testJsonLength() {
@@ -7397,6 +7440,42 @@ public class JdbcTest {
     public MyTable2[] mytable2 = { new MyTable2() };
   }
 
+  /** Locales for which to test DAYNAME and MONTHNAME functions,
+   * and expected results of those functions. */
+  enum TestLocale {
+    ROOT(Locale.ROOT.toString(), shorten("Wednesday"), shorten("Sunday"),
+        shorten("January"), shorten("February")),
+    EN("en", "Wednesday", "Sunday", "January", "February"),
+    FR("fr", "mercredi", "dimanche", "janvier", "f\u00e9vrier"),
+    FR_FR("fr_FR", "mercredi", "dimanche", "janvier", "f\u00e9vrier"),
+    FR_CA("fr_CA", "mercredi", "dimanche", "janvier", "f\u00e9vrier"),
+    ZH_CN("zh_CN", "\u661f\u671f\u4e09", "\u661f\u671f\u65e5", "\u4e00\u6708",
+        "\u4e8c\u6708"),
+    ZH("zh", "\u661f\u671f\u4e09", "\u661f\u671f\u65e5", "\u4e00\u6708",
+        "\u4e8c\u6708");
+
+    private static String shorten(String name) {
+      // In root locale, for Java versions 9 and higher, day and month names
+      // are shortened to 3 letters. This means root locale behaves differently
+      // to English.
+      return TestUtil.getJavaMajorVersion() > 8 ? name.substring(0, 3) : name;
+    }
+
+    public final String localeName;
+    public final String wednesday;
+    public final String sunday;
+    public final String january;
+    public final String february;
+
+    TestLocale(String localeName, String wednesday, String sunday,
+        String january, String february) {
+      this.localeName = localeName;
+      this.wednesday = wednesday;
+      this.sunday = sunday;
+      this.january = january;
+      this.february = february;
+    }
+  }
 }
 
 // End JdbcTest.java
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 8f01cdc..d0a6bc2 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -1511,10 +1511,6 @@ Not implemented:
 | {fn TIMESTAMPADD(timeUnit, count, datetime)} | Adds an interval of *count* *timeUnit*s to a datetime
 | {fn TIMESTAMPDIFF(timeUnit, timestamp1, timestamp2)} | Subtracts *timestamp1* from *timestamp2* and returns the result in *timeUnit*s
 
-Not implemented:
-
-* {fn DAYNAME(date)}
-* {fn MONTHNAME(date)}
 
 #### System
 
@@ -2106,6 +2102,8 @@ semantics.
 | m p | LEFT(string, length)                         | Returns the leftmost *length* characters from the *string*
 | m | TO_BASE64(string)                              | Converts the *string* to base-64 encoded form and returns a encoded string
 | m | FROM_BASE64(string)                            | Returns the decoded result of a base-64 *string* as a string
+| m | {fn DAYNAME(date)}                             | Returns the date of the name of the weekday in a value of datatype DATE; For example, it returns '星期日' for both DATE'2020-02-10' and TIMESTAMP'2020-02-10 10:10:10'
+| m | {fn MONTHNAME(date)}                           | Returns the date of the name of the month in a value of datatype DATE; For example, it returns '二月' for both DATE'2020-02-10' and TIMESTAMP'2020-02-10 10:10:10'
 | o | LTRIM(string)                                  | Returns *string* with all blanks removed from the start
 | o | NVL(value1, value2)                            | Returns *value1* if *value1* is not null, otherwise *value2*
 | m p | REPEAT(string, integer)                      | Returns a string consisting of *string* repeated of *integer* times; returns an empty string if *integer* is less than 1