You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2022/09/08 02:13:16 UTC

[GitHub] [druid] imply-cheddar opened a new pull request, #13052: Expose HTTP Response headers from SqlResource

imply-cheddar opened a new pull request, #13052:
URL: https://github.com/apache/druid/pull/13052

   This change makes the SqlResource expose HTTP response headers in the same way that the QueryResource exposes them.
   
   Fundamentally, the change is to pipe the QueryResponse object all the way through to the Resource so that it can populate response headers.  There is also some code cleanup around DI, as there was a superfluous FactoryFactory class muddying things up.
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966551082


##########
sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java:
##########
@@ -114,4 +114,9 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec)
       writer.endList(frame);
     }
   }
+
+  public static void main(String[] args)
+  {
+    System.out.println(Long.parseLong("9999999999.000000000"));
+  }

Review Comment:
   Yup.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966548543


##########
sql/src/main/java/org/apache/druid/sql/http/SqlResource.java:
##########
@@ -185,20 +178,24 @@ public Response doPost(
                     endLifecycle(stmt, e, os.getCount());
                   }
                 }
-            )
-            .header(SQL_QUERY_ID_RESPONSE_HEADER, sqlQueryId);
-
-        if (sqlQuery.includeHeader()) {
-          responseBuilder.header(SQL_HEADER_RESPONSE_HEADER, SQL_HEADER_VALUE);
-        }
+              }
+          )
+          .header(SQL_QUERY_ID_RESPONSE_HEADER, sqlQueryId);
 
-        return responseBuilder.build();
-      }
-      catch (Throwable e) {
-        // make sure to close yielder if anything happened before starting to serialize the response.
-        yielder0.close();
-        throw new RuntimeException(e);
+      if (sqlQuery.includeHeader()) {
+        responseBuilder.header(SQL_HEADER_RESPONSE_HEADER, SQL_HEADER_VALUE);
       }
+

Review Comment:
   That's code that previously existed and I didn't change.  It just looks like I changed it because of diff stuff.  Look at lines 191-193 of the original.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966554762


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java:
##########
@@ -905,9 +906,10 @@ protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(
         false
     );
 
-    return queryLifecycleFactory
+    final QueryResponse queryResponse = queryLifecycleFactory
         .factorize()
         .runSimple(segmentMetadataQuery, escalator.createEscalatedAuthenticationResult(), Access.OK);
+    return queryResponse == null ? null : queryResponse.getResults();

Review Comment:
   Doh... yeah, it was just a simple mock.  I should've looked at it more.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r967900794


##########
sql/src/main/java/org/apache/druid/sql/SqlStatementFactory.java:
##########
@@ -23,11 +23,30 @@
 
 import javax.servlet.http.HttpServletRequest;
 
-public interface SqlStatementFactory
+public class SqlStatementFactory
 {
-  HttpStatement httpStatement(SqlQuery sqlQuery, HttpServletRequest req);
+  private final SqlToolbox lifecycleToolbox;
 
-  DirectStatement directStatement(SqlQueryPlus sqlRequest);
+  public SqlStatementFactory(SqlToolbox lifecycleToolbox)

Review Comment:
   There is magic in DI platforms, to a certain degree, you want to know the magic to know how things are wired up.
   
   Generally speaking, there is one rule with using Guice that we should be following (I noticed that we have drifted from the rules in some of the code) to help know how things are wired up.  Never, ever, ever, build on or expect implicit bindings (in the code I was looking at, I noticed a number of implicit bindings that were happening, I made explicit ones for everything that I found so far).  We should always have an explicit binding in a module somewhere for everything that we expect to be bound.  The reason for this is that without an explicit binding, it's impossible to do a usages search and figure out how things are setup.  With an explicit binding, you can do a usages search on the class and then find the usages "as a `.class`" which tells you which Module it is bound in.  Provider methods are kinda a gray area here and I generally try to keep away from them if possible.  In this case, without revisiting the object structure in general, either a provider method or an actual
  provider would be required.  For Provider methods, the `.class` usages don't show up, but if you also look for usages as a return value from a method, you can find the Provider methods as well.  Additionally, usages of the constructor will show up here.
   
   TBH, I would also just add the javadoc, but I'm scared of another multi-hour CI cycle blocking the PR.  Maybe I'll do the javadoc and merge anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r967634250


##########
sql/src/main/java/org/apache/druid/sql/guice/SqlModule.java:
##########
@@ -130,4 +137,54 @@ private boolean isAvaticaEnabled()
     Preconditions.checkNotNull(props, "props");
     return Boolean.valueOf(props.getProperty(PROPERTY_SQL_ENABLE_AVATICA, "true"));
   }
+
+  /**
+   * We create a new class for this module so that it can be shared by tests.  The structuring of the SqlModule
+   * at time of writing was not conducive to reuse in test code, so, instead of fixing that we just take the easy
+   * way out of adding the test-reusable code to this module and reuse that.
+   */
+  public static class SqlStatementFactoryModule implements Module

Review Comment:
   Thanks for this. This is some fancy Guice wrangling that makes the setup a bit cleaner.



##########
server/src/main/java/org/apache/druid/server/QueryLifecycle.java:
##########
@@ -151,30 +151,40 @@ public <T> Sequence<T> runSimple(
 
     final Sequence<T> results;
 
+    final QueryResponse queryResponse;
     try {
       preAuthorized(authenticationResult, authorizationResult);
       if (!authorizationResult.isAllowed()) {
         throw new ISE("Unauthorized");
       }
 
-      final QueryResponse queryResponse = execute();
+      queryResponse = execute();
       results = queryResponse.getResults();
     }
     catch (Throwable e) {
       emitLogsAndMetrics(e, null, -1);
       throw e;
     }
 
-    return Sequences.wrap(
-        results,
-        new SequenceWrapper()
-        {
-          @Override
-          public void after(final boolean isDone, final Throwable thrown)
-          {
-            emitLogsAndMetrics(thrown, null, -1);
-          }
-        }
+    /*
+     * It seems extremely weird that the below code is wrapping the Sequence in order to emitLogsAndMetrics.
+     * The Sequence was returned by the call to execute, it would be worthwile to figure out why this wrapping
+     * cannot be moved into execute().  We leave this as an exercise for the future, however as this oddity
+     * was discovered while just trying to expose HTTP response headers

Review Comment:
   Native queries when running from SQL should probably not emit logs and metrics since there is code in the SQL layer to do that: assuming these are basically the same logs and metrics. Are we emitting SQL logs and metrics in SQL, but native ones here? Is that redundant? Should we figure this out?



##########
sql/src/main/java/org/apache/druid/sql/SqlStatementFactory.java:
##########
@@ -23,11 +23,30 @@
 
 import javax.servlet.http.HttpServletRequest;
 
-public interface SqlStatementFactory
+public class SqlStatementFactory
 {
-  HttpStatement httpStatement(SqlQuery sqlQuery, HttpServletRequest req);
+  private final SqlToolbox lifecycleToolbox;
 
-  DirectStatement directStatement(SqlQueryPlus sqlRequest);
+  public SqlStatementFactory(SqlToolbox lifecycleToolbox)

Review Comment:
   Perhaps add a comment to point the reader to the `SqlModule` and `SqlStatementFactoryModule` to explain how these instances are created. It is, unfortunately, not intuitively obvious on the first read how things are now wired up.



##########
sql/src/main/java/org/apache/druid/sql/http/SqlResource.java:
##########
@@ -87,40 +89,28 @@
   private final SqlStatementFactory sqlStatementFactory;
   private final SqlLifecycleManager sqlLifecycleManager;
   private final ServerConfig serverConfig;
+  private final ResponseContextConfig responseContextConfig;
+  private final DruidNode selfNode;
 
   @Inject
-  public SqlResource(
-      @Json ObjectMapper jsonMapper,
-      AuthorizerMapper authorizerMapper,
-      NativeSqlEngine engine,
-      SqlStatementFactoryFactory sqlStatementFactoryFactory,
-      SqlLifecycleManager sqlLifecycleManager,
-      ServerConfig serverConfig
-  )
-  {
-    this(
-        jsonMapper,
-        authorizerMapper,
-        sqlStatementFactoryFactory.factorize(engine),
-        sqlLifecycleManager,
-        serverConfig
-    );
-  }
-
-  @VisibleForTesting
   SqlResource(
       final ObjectMapper jsonMapper,
       final AuthorizerMapper authorizerMapper,
-      final SqlStatementFactory sqlStatementFactory,
+      final @NativeQuery SqlStatementFactory sqlStatementFactory,
       final SqlLifecycleManager sqlLifecycleManager,
-      final ServerConfig serverConfig
+      final ServerConfig serverConfig,
+      ResponseContextConfig responseContextConfig,
+      @Self DruidNode selfNode

Review Comment:
   Nit: `add final` to the above two items just to be consistent? I don't think the Java compiler cares: it can figure out that the items are essentially final. But, we seem to like including the statements as documentation.



##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java:
##########
@@ -341,7 +344,17 @@ public String getFormatString()
         new JoinableFactoryModule(),
         new IndexingServiceTuningConfigModule(),
         new MSQIndexingModule(),
-        new MSQSqlModule(),
+        Modules.override(new MSQSqlModule()).with(
+            binder -> {
+              // Our Guice configuration currently requires bindings to exist even if they aren't ever used, the
+              // following bindings are overriding other bindings that end up needing a lot more dependencies.
+              // We replace the bindings with something that returns null to make things more brittle in case they
+              // actually are used somewhere in the test.
+              binder.bind(SqlStatementFactory.class).annotatedWith(MSQ.class).toProvider(Providers.of(null));

Review Comment:
   This only works if the MSQ tests never use the SQL layer (at least via Guice.) Our Calcite tests are currently a muddle because they hand-build a set of objects, maintained in static variables. An open PR starts to fix this with the ultimate goal of using Guice to create the objects so we don't have to entirely different systems to maintain.
   
   In the branch where the Calcite test refactoring is happening, I ran into issues with the new MSQ tests: they do seem to use Guice, but with objects created in that static bundle of objects. Still trying to sort out that bit of confusion.
   
   The question is then, do the MSQ tests use hand-created objects for the SQL layer or objects from Guice? This change suggests that they are hand-created and the Guice-provided ones are ignored (or we have multiple Guice injectors in play.)
   
   Perhaps the best way to proceed is to commit this, then I'll rebase the Calcite test cleanup on this PR and see what's what. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r967899881


##########
server/src/main/java/org/apache/druid/server/QueryLifecycle.java:
##########
@@ -151,30 +151,40 @@ public <T> Sequence<T> runSimple(
 
     final Sequence<T> results;
 
+    final QueryResponse queryResponse;
     try {
       preAuthorized(authenticationResult, authorizationResult);
       if (!authorizationResult.isAllowed()) {
         throw new ISE("Unauthorized");
       }
 
-      final QueryResponse queryResponse = execute();
+      queryResponse = execute();
       results = queryResponse.getResults();
     }
     catch (Throwable e) {
       emitLogsAndMetrics(e, null, -1);
       throw e;
     }
 
-    return Sequences.wrap(
-        results,
-        new SequenceWrapper()
-        {
-          @Override
-          public void after(final boolean isDone, final Throwable thrown)
-          {
-            emitLogsAndMetrics(thrown, null, -1);
-          }
-        }
+    /*
+     * It seems extremely weird that the below code is wrapping the Sequence in order to emitLogsAndMetrics.
+     * The Sequence was returned by the call to execute, it would be worthwile to figure out why this wrapping
+     * cannot be moved into execute().  We leave this as an exercise for the future, however as this oddity
+     * was discovered while just trying to expose HTTP response headers

Review Comment:
   Which code in the SQL layer handles it?  IIRC, the call on `QueryLifecycle` that this comment is on is special built only for the SQL code to use and I didn't see other code doing the logging, so I guessed that the code I commented on here is actually the only way that the SQL layer handles its logs and metrics.
   
   We should figure it out, but not in this code change :).  I already went sideways doing Guice stuff, don't want to go and expand scope even more ;).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] somu-imply commented on pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
somu-imply commented on PR #13052:
URL: https://github.com/apache/druid/pull/13052#issuecomment-1241312356

   Overall looks good, the Sequence has been changed to a QueryResponse with a separate class and removing the old static QueryResponse class. Most of the places which deal with sequences now has an additional getResult as the return type of execute is now a QueryResponse object. Have one minor comment so far


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cheddar commented on pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
cheddar commented on PR #13052:
URL: https://github.com/apache/druid/pull/13052#issuecomment-1243404359

   I pushed up javadoc fixes and travis got into sad-land from flakey tests :(.  Going to merge anyway, it was a clean build ahead of the javadoc push.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] cheddar merged pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
cheddar merged PR #13052:
URL: https://github.com/apache/druid/pull/13052


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
clintropolis commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966347741


##########
core/src/main/java/org/apache/druid/guice/annotations/MSQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.

Review Comment:
   heh, leftover pasta 🍝 , please update javadocs, this is for `@Self`



##########
core/src/main/java/org/apache/druid/guice/annotations/NativeQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.

Review Comment:
   same comment re: javadoc



##########
server/src/main/java/org/apache/druid/server/security/AuthenticatorMapper.java:
##########
@@ -20,12 +20,10 @@
 package org.apache.druid.server.security;
 
 import com.google.common.collect.Lists;
-import org.apache.druid.guice.ManageLifecycle;
 
 import java.util.List;
 import java.util.Map;
 
-@ManageLifecycle

Review Comment:
   :+1: seems strange this was here



##########
core/src/main/java/org/apache/druid/guice/annotations/NativeQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.
+ * Indicates that the DruidNode bound with this annotation holds the information of the machine where this process
+ * is running.
+ *
+ * @see Parent
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+@PublicApi
+public @interface NativeQ

Review Comment:
   nit naming, if you consider changing `@MSQ` to `@MultiStageQuery` also consider changing this to `@NativeQuery` or `@ClassicQuery`



##########
core/src/main/java/org/apache/druid/guice/annotations/MSQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.
+ * Indicates that the DruidNode bound with this annotation holds the information of the machine where this process
+ * is running.
+ *
+ * @see Parent
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+@PublicApi
+public @interface MSQ

Review Comment:
   also, this is only used in the multi-stage-query extension, should it really be in core and marked `@PublicApi`?



##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java:
##########
@@ -905,9 +906,10 @@ protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(
         false
     );
 
-    return queryLifecycleFactory
+    final QueryResponse queryResponse = queryLifecycleFactory
         .factorize()
         .runSimple(segmentMetadataQuery, escalator.createEscalatedAuthenticationResult(), Access.OK);
+    return queryResponse == null ? null : queryResponse.getResults();

Review Comment:
   it seems like `queryResponse` should never be null?



##########
sql/src/main/java/org/apache/druid/sql/calcite/parser/DruidSqlInsert.java:
##########
@@ -114,4 +114,9 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec)
       writer.endList(frame);
     }
   }
+
+  public static void main(String[] args)
+  {
+    System.out.println(Long.parseLong("9999999999.000000000"));
+  }

Review Comment:
   what is this for? experimenting with something in debugger and forgot to delete?



##########
core/src/main/java/org/apache/druid/guice/annotations/MSQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.
+ * Indicates that the DruidNode bound with this annotation holds the information of the machine where this process
+ * is running.
+ *
+ * @see Parent
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+@PublicApi
+public @interface MSQ

Review Comment:
   nit: any reason not to make this `@MultiStageQuery` or something?



##########
core/src/main/java/org/apache/druid/guice/annotations/NativeQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.
+ * Indicates that the DruidNode bound with this annotation holds the information of the machine where this process
+ * is running.
+ *
+ * @see Parent
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+@PublicApi

Review Comment:
   same question about `@PublicApi`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r967900794


##########
sql/src/main/java/org/apache/druid/sql/SqlStatementFactory.java:
##########
@@ -23,11 +23,30 @@
 
 import javax.servlet.http.HttpServletRequest;
 
-public interface SqlStatementFactory
+public class SqlStatementFactory
 {
-  HttpStatement httpStatement(SqlQuery sqlQuery, HttpServletRequest req);
+  private final SqlToolbox lifecycleToolbox;
 
-  DirectStatement directStatement(SqlQueryPlus sqlRequest);
+  public SqlStatementFactory(SqlToolbox lifecycleToolbox)

Review Comment:
   There is magic in DI platforms, to a certain degree, you must know the magic to know how things are wired up.
   
   Generally speaking, there is one rule with using Guice that we should be following (I noticed that we have drifted from the rules in some of the code) to help know how things are wired up.  Never, ever, ever, build on or expect implicit bindings (in the code I was looking at, I noticed a number of implicit bindings that were happening, I made explicit ones for everything that I found so far).  We should always have an explicit binding in a module somewhere for everything that we expect to be bound.  The reason for this is that without an explicit binding, it's impossible to do a usages search and figure out how things are setup.  With an explicit binding, you can do a usages search on the class and then find the usages "as a `.class`" which tells you which Module it is bound in.  Provider methods are kinda a gray area here and I generally try to keep away from them if possible.  In this case, without revisiting the object structure in general, either a provider method or an actual
  provider would be required.  For Provider methods, the `.class` usages don't show up, but if you also look for usages as a return value from a method, you can find the Provider methods as well.  Additionally, usages of the constructor will show up here.
   
   TBH, I would also just add the javadoc, but I'm scared of another multi-hour CI cycle blocking the PR.  Maybe I'll do the javadoc and merge anyway.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] somu-imply commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
somu-imply commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966482607


##########
sql/src/main/java/org/apache/druid/sql/http/SqlResource.java:
##########
@@ -185,20 +178,24 @@ public Response doPost(
                     endLifecycle(stmt, e, os.getCount());
                   }
                 }
-            )
-            .header(SQL_QUERY_ID_RESPONSE_HEADER, sqlQueryId);
-
-        if (sqlQuery.includeHeader()) {
-          responseBuilder.header(SQL_HEADER_RESPONSE_HEADER, SQL_HEADER_VALUE);
-        }
+              }
+          )
+          .header(SQL_QUERY_ID_RESPONSE_HEADER, sqlQueryId);
 
-        return responseBuilder.build();
-      }
-      catch (Throwable e) {
-        // make sure to close yielder if anything happened before starting to serialize the response.
-        yielder0.close();
-        throw new RuntimeException(e);
+      if (sqlQuery.includeHeader()) {
+        responseBuilder.header(SQL_HEADER_RESPONSE_HEADER, SQL_HEADER_VALUE);
       }
+

Review Comment:
   should this be always added or only in case where includeHeader() is true



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966550933


##########
core/src/main/java/org/apache/druid/guice/annotations/MSQ.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import com.google.inject.BindingAnnotation;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Binding annotation for org.apache.druid.server.DruidNode.
+ * Indicates that the DruidNode bound with this annotation holds the information of the machine where this process
+ * is running.
+ *
+ * @see Parent
+ */
+@Target({ElementType.FIELD, ElementType.PARAMETER, ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+@BindingAnnotation
+@PublicApi
+public @interface MSQ

Review Comment:
   Thanks for calling out my laziness.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r966554342


##########
sql/src/main/java/org/apache/druid/sql/calcite/schema/SegmentMetadataCache.java:
##########
@@ -905,9 +906,10 @@ protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(
         false
     );
 
-    return queryLifecycleFactory
+    final QueryResponse queryResponse = queryLifecycleFactory
         .factorize()
         .runSimple(segmentMetadataQuery, escalator.createEscalatedAuthenticationResult(), Access.OK);
+    return queryResponse == null ? null : queryResponse.getResults();

Review Comment:
   Yeah... one would think, but one of the tests generated an NPE here...
   
   ```
   java.lang.NullPointerException: Cannot invoke "org.apache.druid.server.QueryResponse.getResults()" because "queryResponse" is null
   
   	at org.apache.druid.sql.calcite.schema.SegmentMetadataCache.runSegmentMetadataQuery(SegmentMetadataCache.java:913)
   	at org.apache.druid.sql.calcite.schema.SegmentMetadataCacheTest.testRunSegmentMetadataQueryWithContext(SegmentMetadataCacheTest.java:1096)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:64)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:564)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:59)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:56)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:54)
   	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
   	at org.junit.runners.BlockJUnit4ClassRunner$1.evaluate(BlockJUnit4ClassRunner.java:100)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:366)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:103)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:63)
   	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
   	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
   	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
   	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
   	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
   	at org.junit.runners.Suite.runChild(Suite.java:128)
   	at org.junit.runners.Suite.runChild(Suite.java:27)
   	at org.junit.runners.ParentRunner$4.run(ParentRunner.java:331)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:79)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:329)
   	at org.junit.runners.ParentRunner.access$100(ParentRunner.java:66)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:293)
   	at org.junit.runners.ParentRunner$3.evaluate(ParentRunner.java:306)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:413)
   	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
   	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69)
   	at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:33)
   	at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:235)
   	at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:54)
   ```
   
   I will look a bit more into whether it's just a thing with test setup.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] imply-cheddar commented on a diff in pull request #13052: Expose HTTP Response headers from SqlResource

Posted by GitBox <gi...@apache.org>.
imply-cheddar commented on code in PR #13052:
URL: https://github.com/apache/druid/pull/13052#discussion_r967899456


##########
extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java:
##########
@@ -341,7 +344,17 @@ public String getFormatString()
         new JoinableFactoryModule(),
         new IndexingServiceTuningConfigModule(),
         new MSQIndexingModule(),
-        new MSQSqlModule(),
+        Modules.override(new MSQSqlModule()).with(
+            binder -> {
+              // Our Guice configuration currently requires bindings to exist even if they aren't ever used, the
+              // following bindings are overriding other bindings that end up needing a lot more dependencies.
+              // We replace the bindings with something that returns null to make things more brittle in case they
+              // actually are used somewhere in the test.
+              binder.bind(SqlStatementFactory.class).annotatedWith(MSQ.class).toProvider(Providers.of(null));

Review Comment:
   Yeah... when I was fixing up this test, it was definitely a bit of a mixed bag of strategies.  What I saw is that some of the objects are coming from Guice, but some of the objects are being hand-created.  I had to make this change because the place that I added the binding was used in the test, but the test also didn't have the other dependencies defined to make that "just work", so I had to give Guice just enough to let it make the injector



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org