clintropolis commented on code in PR #18025:
URL: https://github.com/apache/druid/pull/18025#discussion_r2114527521


##########
server/src/test/java/org/apache/druid/server/QueryResourceTest.java:
##########
@@ -528,6 +532,110 @@ public void testSuccessResponseWithTrailerHeader() throws 
IOException
     
Assert.assertEquals(fields.get(QueryResource.RESPONSE_COMPLETE_TRAILER_HEADER), 
"true");
   }
 
+  @Test
+  public void 
testResponseContextContainsMissingSegments_whenLastSegmentIsMissing() throws 
IOException
+  {
+    final SegmentDescriptor missingSegDesc = new SegmentDescriptor(
+        Intervals.of("2025-01-01/P1D"), "0", 1
+    );
+
+    queryResource = new QueryResource(
+        new QueryLifecycleFactory(
+            CONGLOMERATE,
+            new QuerySegmentWalker()
+            {
+              @Override
+              public <T> QueryRunner<T> getQueryRunnerForIntervals(Query<T> 
query, Iterable<Interval> intervals)
+              {
+                return (queryPlus, responseContext) -> new BaseSequence<>(
+                    new BaseSequence.IteratorMaker<T, Iterator<T>>() {
+                      @Override
+                      public Iterator<T> make()
+                      {
+                        List<T> data = Collections.singletonList((T) 
ImmutableMap.of("dummy", 1));
+                        Iterator<T> realIterator = data.iterator();
+
+                        return new Iterator<T>() {
+                          private boolean done = false;
+
+                          @Override
+                          public boolean hasNext()
+                          {
+                            if (realIterator.hasNext()) {
+                              return true;
+                            } else if (!done) {
+                              // Simulate a segment failure in the end after 
initialize() has run
+                              
responseContext.addMissingSegments(ImmutableList.of(missingSegDesc));
+                              done = true;
+                            }
+                            return false;
+                          }
+
+                          @Override
+                          public T next()
+                          {
+                            return realIterator.next();
+                          }
+                        };
+                      }
+
+                      @Override
+                      public void cleanup(Iterator<T> iterFromMake)
+                      {
+                      }
+                    }
+                );
+              }
+
+              @Override
+              public <T> QueryRunner<T> getQueryRunnerForSegments(Query<T> 
query, Iterable<SegmentDescriptor> specs)
+              {
+                throw new UnsupportedOperationException();
+              }
+            },
+            new DefaultGenericQueryMetricsFactory(),
+            new NoopServiceEmitter(),
+            testRequestLogger,
+            new AuthConfig(),
+            NoopPolicyEnforcer.instance(),
+            AuthTestUtils.TEST_AUTHORIZER_MAPPER,
+            Suppliers.ofInstance(new DefaultQueryConfig(ImmutableMap.of()))
+        ),
+        jsonMapper,
+        smileMapper,
+        queryScheduler,
+        new AuthConfig(),
+        null,
+        ResponseContextConfig.newConfig(true),
+        DRUID_NODE
+    );
+
+    expectPermissiveHappyPathAuth();
+
+    org.eclipse.jetty.server.Response response = 
this.jettyResponseforRequest(testServletRequest);
+
+    // Execute the query
+    Assert.assertNull(queryResource.doPost(
+        new 
ByteArrayInputStream(SIMPLE_TIMESERIES_QUERY.getBytes(StandardCharsets.UTF_8)),
+        null,
+        testServletRequest
+    ));
+
+
+    Assert.assertTrue(response.containsHeader(HttpHeader.TRAILER.toString()));
+    Assert.assertEquals(QueryResultPusher.RESULT_TRAILER_HEADERS, 
response.getHeader(HttpHeader.TRAILER.toString()));
+
+    final HttpFields observedFields = response.getTrailers().get();
+
+    
Assert.assertTrue(response.containsHeader(QueryResource.HEADER_RESPONSE_CONTEXT));
+    final DataServerResponse expectedResponse = new 
DataServerResponse(ImmutableList.of(missingSegDesc));

Review Comment:
   it feels kind of weird to use `DataServerResponse` since `DataServerClient` 
is mainly an MSQ thing, but i suppose is the same shape as the response context 
here... I don't feel too strongly either way, so i suppose is fine to leave 
like this or like just serialize a map since like the `DataServerResponse` is 
just being used to serialize to a string? :shrug:



##########
integration-tests/src/main/java/org/apache/druid/server/coordination/ServerManagerForQueryErrorTest.java:
##########
@@ -59,14 +59,21 @@
 
 /**
  * This server manager is designed to test various query failures.
- *
- * - Missing segments. A segment can be missing during a query if a historical 
drops the segment
+ * <ul>
+ *  <li> Missing segments. A segment can be missing during a query if a 
historical drops the segment

Review Comment:
   since ITs are in a kind of weird state where some of them are using the old 
test framework (the modifications you've done in this PR) and some are in the 
new test framework (integration-tests-ex/), there is a duplicate version of 
this file in 
https://github.com/apache/druid/blob/master/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/ServerManagerForQueryErrorTest.java,
 i think it would be good to keep them synchronized. These query retry tests 
have not yet been migrated over to the new framework, so i think its fine to 
add the tests here, just thinking ahead for if we do ever finish migrating them 
(which i hope happens someday so at least the tests are consistent)



##########
server/src/main/java/org/apache/druid/server/QueryResultPusher.java:
##########
@@ -404,42 +404,10 @@ public void initialize()
 
         DirectDruidClient.removeMagicResponseContextFields(responseContext);
 
-        // Limit the response-context header, see 
https://github.com/apache/druid/issues/2331
-        // Note that Response.ResponseBuilder.header(String key,Object 
value).build() calls value.toString()
-        // and encodes the string using ASCII, so 1 char is = 1 byte
-        ResponseContext.SerializationResult serializationResult;
-        try {
-          serializationResult = responseContext.serializeWith(
-              jsonMapper,
-              responseContextConfig.getMaxResponseContextHeaderSize()
-          );
-        }
-        catch (JsonProcessingException e) {
-          log.info(e, "Problem serializing to JSON!?");
-          serializationResult = new ResponseContext.SerializationResult("Could 
not serialize", "Could not serialize");
-        }
-
-        if (serializationResult.isTruncated()) {
-          final String logToPrint = StringUtils.format(
-              "Response Context truncated for id [%s]. Full context is [%s].",
-              queryId,
-              serializationResult.getFullResult()
-          );
-          if (responseContextConfig.shouldFailOnTruncatedResponseContext()) {
-            log.error(logToPrint);
-            throw new QueryInterruptedException(
-                new TruncatedResponseContextException(
-                    "Serialized response context exceeds the max size[%s]",
-                    responseContextConfig.getMaxResponseContextHeaderSize()
-                ),
-                selfNode.getHostAndPortToUse()
-            );
-          } else {
-            log.warn(logToPrint);
-          }
-        }
+        // validate the response context early to fail-fast, but don’t write 
it to the response yet,
+        // as additional things may still be accumulated.
+        serializeAndValidateResponseContextHeader();

Review Comment:
   is there any reason to not leave the `response.setHeader` here just in case? 
Per the javadocs, "If the header had already been set, the new value overwrites 
the previous one."



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to