-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changing the array behavior to start the index from 1. #1682
Changes from all commits
472287f
c953aaa
b63959f
eb7034f
7f31600
501f0d1
79cc7f2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -166,13 +166,18 @@ The supported column data types are: | |
- ``BIGINT`` | ||
- ``DOUBLE`` | ||
- ``VARCHAR`` (or ``STRING``) | ||
- ``ARRAY<ArrayType>`` (JSON and AVRO only. Index starts from 0) | ||
- ``ARRAY<ArrayType>`` (JSON and AVRO only. Index starting from 0 but this is configurable. Refer to below for more details.) | ||
- ``MAP<VARCHAR, ValueType>`` (JSON and AVRO only) | ||
- ``STRUCT<FieldName FieldType, ...>`` (JSON and AVRO only) The STRUCT type requires you to specify a list of fields. | ||
For each field you must specify the field name (FieldName) and field type (FieldType). The field type can be any of | ||
the supported KSQL types, including the complex types ``MAP``, ``ARRAY``, and ``STRUCT``. ``STRUCT`` fields can be | ||
accessed in expressions using the struct dereference (``->``) operator. See :ref:`operators` for more details. | ||
|
||
You can configure the base index for arrays to start from 0 or 1 using ``ksql.functions.array.legacy.base`` configuration value. | ||
The current default value of ``ksql.functions.array.legacy.base`` is true indicating that the array indexes start from 0. | ||
You can set ``ksql.functions.array.legacy.base`` to false in order to have array indexes startinhg from 1. | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. typo: starting |
||
We plan to set the default base for array indexes to 1 in future releases. | ||
|
||
KSQL adds the implicit columns ``ROWTIME`` and ``ROWKEY`` to every | ||
stream and table, which represent the corresponding Kafka message | ||
timestamp and message key, respectively. The timestamp has milliseconds accuracy. | ||
|
@@ -243,13 +248,18 @@ The supported column data types are: | |
- ``BIGINT`` | ||
- ``DOUBLE`` | ||
- ``VARCHAR`` (or ``STRING``) | ||
- ``ARRAY<ArrayType>`` (JSON and AVRO only. Index starts from 0) | ||
- ``ARRAY<ArrayType>`` (JSON and AVRO only. Index starting from 0 but this is configurable. Refer to below for more details.) | ||
- ``MAP<VARCHAR, ValueType>`` (JSON and AVRO only) | ||
- ``STRUCT<FieldName FieldType, ...>`` (JSON and AVRO only) The STRUCT type requires you to specify a list of fields. | ||
For each field you must specify the field name (FieldName) and field type (FieldType). The field type can be any of | ||
the supported KSQL types, including the complex types ``MAP``, ``ARRAY``, and ``STRUCT``. ``STRUCT`` fields can be | ||
accessed in expressions using the struct dereference (``->``) operator. See :ref:`operators` for more details. | ||
|
||
You can configure the base index for arrays to start from 0 or 1 using ``ksql.functions.array.legacy.base`` configuration value. | ||
The current default value of ``ksql.functions.array.legacy.base`` is true indicating that the array indexes start from 0. | ||
You can set ``ksql.functions.array.legacy.base`` to false in order to have array indexes startinhg from 1. | ||
We plan to set the default base for array indexes to 1 in future releases. | ||
|
||
KSQL adds the implicit columns ``ROWTIME`` and ``ROWKEY`` to every | ||
stream and table, which represent the corresponding Kafka message | ||
timestamp and message key, respectively. The timestamp has milliseconds accuracy. | ||
|
@@ -988,6 +998,11 @@ The explanation for each operator includes a supporting example based on the fol | |
- Subscript (``[subscript_expr]``) The subscript operator is used to reference the value at | ||
an array index or a map key. | ||
|
||
You can configure the base index for arrays to start from 0 or 1 using ``ksql.functions.array.legacy.base`` configuration value. | ||
The current default value of ``ksql.functions.array.legacy.base`` is true indicating that the array indexes start from 0. | ||
You can set ``ksql.functions.array.legacy.base`` to false in order to have array indexes startinhg from 1. | ||
We plan to set the default base for array indexes to 1 in future releases. | ||
|
||
.. code:: sql | ||
|
||
SELECT NICKNAMES[0] FROM USERS; | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -54,6 +54,7 @@ | |
import io.confluent.ksql.parser.tree.SubscriptExpression; | ||
import io.confluent.ksql.parser.tree.SymbolReference; | ||
import io.confluent.ksql.util.ExpressionTypeManager; | ||
import io.confluent.ksql.util.KsqlConfig; | ||
import io.confluent.ksql.util.KsqlException; | ||
import io.confluent.ksql.util.Pair; | ||
import io.confluent.ksql.util.SchemaUtil; | ||
|
@@ -66,14 +67,19 @@ | |
|
||
public class SqlToJavaVisitor { | ||
|
||
private Schema schema; | ||
private FunctionRegistry functionRegistry; | ||
private final Schema schema; | ||
private final FunctionRegistry functionRegistry; | ||
private final KsqlConfig ksqlConfig; | ||
|
||
private final ExpressionTypeManager expressionTypeManager; | ||
|
||
public SqlToJavaVisitor(final Schema schema, final FunctionRegistry functionRegistry) { | ||
public SqlToJavaVisitor( | ||
final Schema schema, | ||
final FunctionRegistry functionRegistry, | ||
final KsqlConfig ksqlConfig) { | ||
this.schema = schema; | ||
this.functionRegistry = functionRegistry; | ||
this.ksqlConfig = ksqlConfig; | ||
this.expressionTypeManager = | ||
new ExpressionTypeManager(schema, functionRegistry); | ||
} | ||
|
@@ -539,11 +545,12 @@ protected Pair<String, Schema> visitSubscriptExpression( | |
switch (internalSchema.type()) { | ||
case ARRAY: | ||
return new Pair<>( | ||
String.format("((%s) ((%s)%s).get((int)(%s)))", | ||
String.format("((%s) (ArrayGet.getItem(((%s) %s), (int)(%s), %s)))", | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If index is null, then this will throw NPE. The index may be derived from the source query e.g.
So we may want to consider handling a null index by just returning null. (I've not checked what other vendors do, but most seem to err on the side of not-logging-loads-of-errors and just returning null) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. One problem with returning null is that we won't know if the value of the array item is null or there has been an error. We do return null for the column in case of exception anyway but I think having the error cause in the log is informative, although I agree that we may end up having lot's of errors in the log. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
I'd normally agree with you. I'm a fan or fail hard and fail fast. However, the pattern for most SQL implementations is to be pretty lax about error reporting. For example, substring just returns null if the start index is out of bounds. At the very least, we should be logging a more informative error than an NPE. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I commented in another thread, but I think it would be better form to log the exception explicitly at a debug level so that we can control what goes into our logs. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we throw an exception it will be logged in the processing log... |
||
SchemaUtil.getJavaType(internalSchema.valueSchema()).getSimpleName(), | ||
internalSchemaJavaType, | ||
process(node.getBase(), unmangleNames).getLeft(), | ||
process(node.getIndex(), unmangleNames).getLeft() | ||
process(node.getIndex(), unmangleNames).getLeft(), | ||
ksqlConfig.getBoolean(KsqlConfig.KSQL_FUNCTIONS_ARRAY_LEGACY_BASE_CONFIG) | ||
), | ||
internalSchema.valueSchema() | ||
); | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* Copyright 2018 Confluent Inc. | ||
* | ||
* Licensed under the Apache License, Version 2.0 (the "License"); | ||
* you may not use this file except in compliance with the License. | ||
* You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
**/ | ||
|
||
package io.confluent.ksql.codegen.helpers; | ||
|
||
import java.util.List; | ||
|
||
public final class ArrayGet { | ||
|
||
private ArrayGet() {} | ||
|
||
public static Object getItem(final List<?> array, final int index, final boolean isLegacy) { | ||
final int correctIndex = isLegacy ? index : index - 1; | ||
return array.get(correctIndex); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You've removed the out of bounds handing that was here before. Just wondering why? I'm guessing this is inline with your comment above, and you want the exception to be logged? If so, then I think this should be a general question to the team to come to a decision on how we handle such issues: do we silently handle them and return null, or do we throw an exception, log it and then return null? Personally, I'm with you and prefer the logging approach. But if this is not what expected by our users then they might not thank us for spamming the logs. Likewise, in cloud we're going to end up with a lot of noise in the logs, (though we can turn it off I guess). Which ever way we decide to go we should be consistent. So, for example, if we go with throwing exceptions, then we should change There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think for now we can throw an exception and return null. This way we make sure we don't lose information that user may need. This is the same behavior for other types of expressions. Let's make There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not a fan of just dumping exception traces to the log by default. Even if we have #1795, it would be much better to explicitly log this at some sort of verbose logging level (like debug) so that we can control what is logged by setting the class level log level in the config. If these types of things were logged at debug level, then in most cases we won't see them until we want to debug something, at which point we can change the log level for the class (potentially at run time through JMX) and then collect the necessary information. If you see our soak cluster, we already have many NPE stack traces in the logs. These are really hard to work with because:
I vote for not adding to the problem in this patch. |
||
} | ||
|
||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -53,12 +53,18 @@ | |
import java.util.stream.Collectors; | ||
import org.apache.kafka.connect.data.Schema; | ||
import org.apache.kafka.connect.data.SchemaBuilder; | ||
import org.easymock.EasyMock; | ||
import org.easymock.EasyMockRunner; | ||
import org.easymock.Mock; | ||
import org.easymock.MockType; | ||
import org.junit.Assert; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
import org.junit.runner.RunWith; | ||
|
||
|
||
@SuppressWarnings("SameParameterValue") | ||
@RunWith(EasyMockRunner.class) | ||
public class CodeGenRunnerTest { | ||
|
||
private static final int INT64_INDEX1 = 0; | ||
|
@@ -80,6 +86,10 @@ public class CodeGenRunnerTest { | |
private final InternalFunctionRegistry functionRegistry = new InternalFunctionRegistry(); | ||
private GenericRowValueTypeEnforcer genericRowValueTypeEnforcer; | ||
private final KsqlConfig ksqlConfig = new KsqlConfig(Collections.emptyMap()); | ||
private Schema schema; | ||
|
||
@Mock(type = MockType.NICE) | ||
KsqlConfig localKsqlConfig; | ||
|
||
@Before | ||
public void init() { | ||
|
@@ -90,7 +100,7 @@ public void init() { | |
final Schema arraySchema = SchemaBuilder.array(Schema.STRING_SCHEMA).optional().build(); | ||
|
||
|
||
final Schema schema = SchemaBuilder.struct() | ||
schema = SchemaBuilder.struct() | ||
.field("CODEGEN_TEST.COL0", SchemaBuilder.OPTIONAL_INT64_SCHEMA) | ||
.field("CODEGEN_TEST.COL1", SchemaBuilder.OPTIONAL_STRING_SCHEMA) | ||
.field("CODEGEN_TEST.COL2", SchemaBuilder.OPTIONAL_STRING_SCHEMA) | ||
|
@@ -158,6 +168,7 @@ public void testIsDistinctFrom() throws Exception { | |
|
||
@Test | ||
public void testIsNull() throws Exception { | ||
// Given: | ||
final String simpleQuery = "SELECT col0 IS NULL FROM CODEGEN_TEST;"; | ||
final Analysis analysis = analyzeQuery(simpleQuery, metaStore); | ||
|
||
|
@@ -179,7 +190,7 @@ public void testIsNull() throws Exception { | |
|
||
@Test | ||
public void shouldHandleMultiDimensionalArray() throws Exception { | ||
final String simpleQuery = "SELECT col14[0][0] FROM CODEGEN_TEST;"; | ||
final String simpleQuery = "SELECT col14[1][1] FROM CODEGEN_TEST;"; | ||
final Analysis analysis = analyzeQuery(simpleQuery, metaStore); | ||
final ExpressionMetadata expressionEvaluatorMetadata = codeGenRunner.buildCodeGenFromParseTree | ||
(analysis.getSelectExpressions().get(0)); | ||
|
@@ -191,6 +202,36 @@ public void shouldHandleMultiDimensionalArray() throws Exception { | |
assertThat(result, equalTo("item_11")); | ||
} | ||
|
||
@Test | ||
public void shouldHandleMultiDimensionalArrayWithBaseStartingFrom1() throws Exception { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit, spell out the |
||
// Given: | ||
|
||
EasyMock.expect(localKsqlConfig.getBoolean(EasyMock.eq(KsqlConfig.KSQL_FUNCTIONS_ARRAY_LEGACY_BASE_CONFIG))).andReturn(false).times(2); | ||
EasyMock.replay(localKsqlConfig); | ||
final CodeGenRunner localCodeGenRunner = new CodeGenRunner(schema, localKsqlConfig, functionRegistry); | ||
final String simpleQuery = "SELECT col14[1][1] FROM CODEGEN_TEST;"; | ||
final Analysis analysis = analyzeQuery(simpleQuery, metaStore); | ||
|
||
final Object[] arguments = {Arrays.asList( | ||
Arrays.asList("item_11", "item_12"), | ||
Arrays.asList("item_21", "item_22"))}; | ||
|
||
// When: | ||
final Object result = executeExpression(simpleQuery, arguments, localCodeGenRunner); | ||
|
||
// Then: | ||
EasyMock.verify(localKsqlConfig); | ||
assertThat(result, instanceOf(String.class)); | ||
assertThat(result, equalTo("item_11")); | ||
} | ||
|
||
private Object executeExpression(final String simpleQuery, final Object[] arguments, final CodeGenRunner localCodeGenRunner) throws Exception { | ||
final Analysis analysis = analyzeQuery(simpleQuery, metaStore); | ||
final ExpressionMetadata expressionEvaluatorMetadata = localCodeGenRunner.buildCodeGenFromParseTree | ||
(analysis.getSelectExpressions().get(0)); | ||
return expressionEvaluatorMetadata.getExpressionEvaluator().evaluate(arguments); | ||
} | ||
|
||
@Test | ||
public void testIsNotNull() throws Exception { | ||
final String simpleQuery = "SELECT col0 IS NOT NULL FROM CODEGEN_TEST;"; | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why can't we just name this
ksql.functions.array.zero.base
. It seems like a much more self explanatory name.