leventov commented on a change in pull request #8157: Enum of ResponseContext 
keys
URL: https://github.com/apache/incubator-druid/pull/8157#discussion_r309838142
 
 

 ##########
 File path: 
processing/src/main/java/org/apache/druid/query/context/ResponseContext.java
 ##########
 @@ -236,95 +253,124 @@ public static ResponseContext createEmpty()
     return DefaultResponseContext.createEmpty();
   }
 
+  /**
+   * Deserializes a string into {@link ResponseContext} using given {@link 
ObjectMapper}.
+   * @throws IllegalStateException if one of the deserialized map keys has not 
been registered.
+   */
   public static ResponseContext deserialize(String responseContext, 
ObjectMapper objectMapper) throws IOException
   {
-    final Map<String, Object> delegate = objectMapper.readValue(
+    final Map<String, Object> keyNameToObjects = objectMapper.readValue(
         responseContext,
         JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
     );
-    return new ResponseContext()
-    {
-      @Override
-      protected Map<String, Object> getDelegate()
-      {
-        return delegate;
-      }
-    };
+    final ResponseContext context = ResponseContext.createEmpty();
+    keyNameToObjects.forEach((keyName, value) -> {
+      final BaseKey key = Key.keyOf(keyName);
+      context.add(key, value);
+    });
+    return context;
   }
 
-  protected abstract Map<String, Object> getDelegate();
+  protected abstract Map<BaseKey, Object> getDelegate();
 
+  /**
+   * Associates the specified object with the specified key.
+   * @throws IllegalStateException if the key has not been registered.
+   */
   public Object put(BaseKey key, Object value)
   {
-    return getDelegate().put(key.getName(), value);
+    final BaseKey registeredKey = Key.keyOf(key.getName());
+    return getDelegate().put(registeredKey, value);
   }
 
   public Object get(BaseKey key)
   {
-    return getDelegate().get(key.getName());
+    return getDelegate().get(key);
   }
 
   public Object remove(BaseKey key)
   {
-    return getDelegate().remove(key.getName());
+    return getDelegate().remove(key);
   }
 
   /**
    * Adds (merges) a new value associated with a key to an old value.
    * See merge function of a context key for a specific implementation.
+   * @throws IllegalStateException if the key has not been registered.
    */
   public Object add(BaseKey key, Object value)
   {
-    return getDelegate().merge(key.getName(), value, key.getMergeFunction());
+    final BaseKey registeredKey = Key.keyOf(key.getName());
+    return getDelegate().merge(registeredKey, value, key.getMergeFunction());
   }
 
   /**
-   * Merges a response context into current.
-   * This method merges only keys from the enum {@link Key}.
+   * Merges a response context into the current.
+   * @throws IllegalStateException If a key of the {@code responseContext} has 
not been registered.
    */
   public void merge(ResponseContext responseContext)
   {
-    for (BaseKey key : Key.getKeys()) {
-      final Object newValue = responseContext.get(key);
+    responseContext.getDelegate().forEach((key, newValue) -> {
       if (newValue != null) {
         add(key, newValue);
       }
-    }
+    });
   }
 
   /**
    * Serializes the context given that the resulting string length is less 
than the provided limit.
-   * The method removes max-length fields one by one if the resulting string 
length is greater than the limit.
-   * The resulting string might be correctly deserialized as a {@link 
ResponseContext}.
+   * This method tries to remove some elements from context collections if 
it's needed to satisfy the limit.
+   * The resulting string might be correctly deserialized to {@link 
ResponseContext}.
    */
   public SerializationResult serializeWith(ObjectMapper objectMapper, int 
maxCharsNumber) throws JsonProcessingException
   {
     final String fullSerializedString = 
objectMapper.writeValueAsString(getDelegate());
     if (fullSerializedString.length() <= maxCharsNumber) {
       return new SerializationResult(fullSerializedString, 
fullSerializedString);
     } else {
-      final HashMap<String, Object> copiedMap = new HashMap<>(getDelegate());
-      final PriorityQueue<Map.Entry<String, String>> serializedValueEntries = 
new PriorityQueue<>(
-          Comparator.comparing((Map.Entry<String, String> e) -> 
e.getValue().length()).reversed()
-      );
-      for (Map.Entry<String, Object> e : copiedMap.entrySet()) {
-        serializedValueEntries.add(new AbstractMap.SimpleImmutableEntry<>(
-            e.getKey(),
-            objectMapper.writeValueAsString(e.getValue())
-        ));
-      }
-      // quadratic complexity: while loop with map serialization on each 
iteration
-      while (!copiedMap.isEmpty() && !serializedValueEntries.isEmpty()) {
-        final Map.Entry<String, String> maxLengthEntry = 
serializedValueEntries.poll();
-        Preconditions.checkNotNull(maxLengthEntry);
-        copiedMap.remove(maxLengthEntry.getKey());
-        final String reducedSerializedString = 
objectMapper.writeValueAsString(copiedMap);
-        if (reducedSerializedString.length() <= maxCharsNumber) {
-          return new SerializationResult(reducedSerializedString, 
fullSerializedString);
+      // Indicates that the context is truncated during serialization.
+      add(Key.TRUNCATED, true);
+      final ObjectNode contextJsonNode = 
objectMapper.valueToTree(getDelegate());
+      final ArrayList<Map.Entry<String, JsonNode>> sortedNodesByLength = 
Lists.newArrayList(contextJsonNode.fields());
+      final Comparator<Map.Entry<String, JsonNode>> 
valueLengthReversedComparator =
+          Comparator.comparing((Map.Entry<String, JsonNode> e) -> 
e.getValue().toString().length()).reversed();
+      sortedNodesByLength.sort(valueLengthReversedComparator);
+      int needToRemoveCharsNumber = fullSerializedString.length() - 
maxCharsNumber;
+      // The complexity of this block is O(n*m*log(m)) where n - context size, 
m - context's array size
+      for (Map.Entry<String, JsonNode> e : sortedNodesByLength) {
+        final String fieldName = e.getKey();
+        final JsonNode node = e.getValue();
+        if (node.isArray()) {
+          if (needToRemoveCharsNumber >= node.toString().length()) {
+            final int lengthBeforeRemove = node.toString().length();
+            // Empty array could be correctly deserialized so we remove only 
its elements.
 
 Review comment:
   I think the logic of this block should avoid producing empty array because 
it may be misleading.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

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

Reply via email to