You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by "noblepaul (via GitHub)" <gi...@apache.org> on 2023/05/22 08:16:32 UTC

[GitHub] [solr] noblepaul opened a new pull request, #1655: SOLR-16812: Support CBOR format for update/query

noblepaul opened a new pull request, #1655:
URL: https://github.com/apache/solr/pull/1655

   (no comment)


-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] gerlowskija commented on a diff in pull request #1655: SOLR-16812: Support CBOR format for update/query

Posted by "gerlowskija (via GitHub)" <gi...@apache.org>.
gerlowskija commented on code in PR #1655:
URL: https://github.com/apache/solr/pull/1655#discussion_r1218300923


##########
solr/core/src/test/org/apache/solr/util/TestCborDataFormat.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.solr.util;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.loader.CborLoader;
+import org.apache.solr.response.XMLResponseWriter;
+
+public class TestCborDataFormat extends SolrCloudTestCase {
+
+  @SuppressWarnings("unchecked")
+  public void testRoundTrip() throws Exception {
+    String testCollection = "testRoundTrip";
+
+    MiniSolrCloudCluster cluster =
+        configureCluster(1)
+            .addConfig(
+                "conf", TEST_PATH().resolve("configsets").resolve("cloud-managed").resolve("conf"))
+            .configure();
+    try {
+      System.setProperty("managed.schema.mutable", "true");
+      CloudSolrClient client = cluster.getSolrClient();
+      CollectionAdminRequest.createCollection(testCollection, "conf", 1, 1).process(client);
+      modifySchema(testCollection, client);
+
+      byte[] b =
+          Files.readAllBytes(
+              new File(ExternalPaths.SOURCE_HOME, "example/films/films.json").toPath());
+      // every operation is performed twice. We should only take the second number
+      // so that we give JVM a chance to optimize that code
+      index(testCollection, client, createJsonReq(b), true);
+      index(testCollection, client, createJsonReq(b), true);
+
+      index(testCollection, client, createJavabinReq(b), true);
+      index(testCollection, client, createJavabinReq(b), true);
+
+      index(testCollection, client, createCborReq(b), true);
+      index(testCollection, client, createCborReq(b), false);
+
+      runQuery(testCollection, client, "javabin");
+      runQuery(testCollection, client, "javabin");
+      runQuery(testCollection, client, "json");
+      runQuery(testCollection, client, "json");
+      b = runQuery(testCollection, client, "cbor");
+      b = runQuery(testCollection, client, "cbor");
+      ObjectMapper objectMapper = new ObjectMapper(new CBORFactory());
+      Object o = objectMapper.readValue(b, Object.class);
+      List<Object> l = (List<Object>) Utils.getObjectByPath(o, false, "response/docs");
+      assertEquals(1100, l.size());
+    } finally {
+      System.clearProperty("managed.schema.mutable");
+      cluster.shutdown();
+    }
+  }
+
+  private void index(
+      String testCollection, CloudSolrClient client, GenericSolrRequest r, boolean del)
+      throws Exception {
+    RTimer timer = new RTimer();
+    client.request(r, testCollection);
+    System.out.println("INDEX_TIME: " + r.contentWriter.getContentType() + " : " + timer.getTime());
+    if (del) {
+      UpdateRequest req = new UpdateRequest().deleteByQuery("*:*");
+      req.setParam("commit", "true");
+      client.request(req, testCollection);
+    }
+  }
+
+  private byte[] runQuery(String testCollection, CloudSolrClient client, String wt)
+      throws SolrServerException, IOException {
+    NamedList<Object> result;
+    QueryRequest request;
+    RTimer timer = new RTimer();
+    request = new QueryRequest(new SolrQuery("*:*").setRows(1111));
+    request.setResponseParser(new InputStreamResponseParser(wt));
+    result = client.request(request, testCollection);
+    byte[] b = copyStream((InputStream) result.get("stream"));
+    System.out.println(wt + "_time : " + timer.getTime());

Review Comment:
   [Q] So, as I read this code - `runQuery` will capture CBOR serialization in its perf numbers but **not** deserialization since we're not deserializing the received data on the client side.
   
   Am I reading that right?  And if so, is that intentional?
   
   [0] Also, doesn't forbiddenApis complain about the `System.out.println` usage here?  I swear that was one of the things we disallowed, though maybe we're looser about that in our tests...



##########
solr/core/src/test/org/apache/solr/util/TestCborDataFormat.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.solr.util;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.loader.CborLoader;
+import org.apache.solr.response.XMLResponseWriter;
+
+public class TestCborDataFormat extends SolrCloudTestCase {
+
+  @SuppressWarnings("unchecked")
+  public void testRoundTrip() throws Exception {
+    String testCollection = "testRoundTrip";
+
+    MiniSolrCloudCluster cluster =
+        configureCluster(1)
+            .addConfig(
+                "conf", TEST_PATH().resolve("configsets").resolve("cloud-managed").resolve("conf"))
+            .configure();
+    try {
+      System.setProperty("managed.schema.mutable", "true");
+      CloudSolrClient client = cluster.getSolrClient();
+      CollectionAdminRequest.createCollection(testCollection, "conf", 1, 1).process(client);
+      modifySchema(testCollection, client);
+
+      byte[] b =
+          Files.readAllBytes(
+              new File(ExternalPaths.SOURCE_HOME, "example/films/films.json").toPath());
+      // every operation is performed twice. We should only take the second number

Review Comment:
   [-0] Is this class a test meant to validate correctness, or a benchmark for measuring performance?  I think both are needed and valuable; it's great you thought of both.  But I think it's a mistake to combine the two in a single class.  Especially when we have a whole "benchmark" module built for exactly the sort of measurements you're taking.  It offers tools (e.g. JMH annotations, data generators) that make it easy to test performance much more robustly than is feasible to do here.  (See my comment below).
   
   [-1] It's a good start, but I think there's a handful of problems with how this JUnit method gathers numbers around CBOR queries, that make it misleading for gathering even anecdotal performance data.
   
   I'm sure you've done other benchmarking at this point to prove out the CBOR idea, but so far these are the only numbers the rest of the community has seen.  So we need to make sure they're rock solid!  In particular, here are a few of my concerns:
   
   1. Even for tiny methods called in a tight loop, JVMs often take more than n=2 iterations to optimize bytecode.  The 'N' here is just too small to assume the JVM is fully "hot" on the ser-de code for each format.  Even ignoring JVM optimizations, n=2 is too small to rule out random noise (other processes competing for CPU, JVM GC, etc.) on the host machine.
   2. The order in which you're testing formats likely skews results.  e.g. The two JSON queries run on an almost entirely cold JVM, whereas CBOR queries run when the JVM, OS caches, Solr caches, etc. are considerably warmer.
   3. "films.json" is pretty small as a data set and homogenous in terms of a few field-types.  I worry that would make the gathered data unrepresentative and noisy.
   
   I think all three of these would be easy to address if the code was moved to the benchmarking module.  It has data generators for building larger and more diverse datasets.  JMH has a lot of support built in that make it easy to configure "Warming" iterations that get ignored in the collected statistics. etc.



##########
solr/core/src/test/org/apache/solr/util/TestCborDataFormat.java:
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.solr.util;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.loader.CborLoader;
+import org.apache.solr.response.XMLResponseWriter;
+
+public class TestCborDataFormat extends SolrCloudTestCase {
+
+  @SuppressWarnings("unchecked")
+  public void testRoundTrip() throws Exception {
+    String testCollection = "testRoundTrip";
+
+    MiniSolrCloudCluster cluster =
+        configureCluster(1)
+            .addConfig(
+                "conf", TEST_PATH().resolve("configsets").resolve("cloud-managed").resolve("conf"))
+            .configure();
+    try {
+      System.setProperty("managed.schema.mutable", "true");
+      CloudSolrClient client = cluster.getSolrClient();
+      CollectionAdminRequest.createCollection(testCollection, "conf", 1, 1).process(client);
+      modifySchema(testCollection, client);
+
+      byte[] b =
+          Files.readAllBytes(
+              new File(ExternalPaths.SOURCE_HOME, "example/films/films.json").toPath());

Review Comment:
   [Q] Does the films dataset have sufficient variety of field types to fully validate the cbor serialization/deserialization logic?  Is there testing elsewhere that would make sure that cbor serialization doesn't blow up when presented with e.g. a 'binary' or 'pint' field



-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] mkhludnev commented on a diff in pull request #1655: SOLR-16812: Support CBOR format for update/query

Posted by "mkhludnev (via GitHub)" <gi...@apache.org>.
mkhludnev commented on code in PR #1655:
URL: https://github.com/apache/solr/pull/1655#discussion_r1205464743


##########
solr/solrj/src/java/org/apache/solr/client/solrj/request/GenericSolrRequest.java:
##########
@@ -42,6 +44,22 @@ public GenericSolrRequest setContentWriter(ContentWriter contentWriter) {
     return this;
   }
 
+  public GenericSolrRequest withContent(byte[] buf, String type) {

Review Comment:
   Isn't it too harmful to be advertised as public for users?



-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a diff in pull request #1655: SOLR-16812: Support CBOR format for update/query

Posted by "dsmiley (via GitHub)" <gi...@apache.org>.
dsmiley commented on code in PR #1655:
URL: https://github.com/apache/solr/pull/1655#discussion_r1210896349


##########
solr/core/src/test/org/apache/solr/util/TestCborDataFormat.java:
##########
@@ -0,0 +1,228 @@
+/*
+ * 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.solr.util;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.LongAdder;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.InputStreamResponseParser;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.GenericSolrRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.RequestWriter;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.cloud.MiniSolrCloudCluster;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.params.MapSolrParams;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.loader.CborStream;
+import org.apache.solr.response.XMLResponseWriter;
+
+public class TestCborDataFormat extends SolrCloudTestCase {

Review Comment:
   IMO our request/response formats, to include this new one, should be tested in a general way without a dedicated test, especially not a SolrCloud test.  For example, we randomly pick compatible substitutes for the codec.  Shouldn't we do the same for the underlying "wt" encoding too?



##########
solr/core/src/java/org/apache/solr/handler/loader/CborStream.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.solr.handler.loader;
+
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import com.fasterxml.jackson.dataformat.cbor.CBORParser;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+
+public class CborStream {

Review Comment:
   javadoc on public classes



##########
solr/core/src/java/org/apache/solr/handler/UpdateRequestHandler.java:
##########
@@ -146,6 +147,7 @@ protected Map<String, ContentStreamLoader> createDefaultLoaders(NamedList<?> arg
     registry.put("application/json", new JsonLoader().init(p));
     registry.put("application/csv", new CSVLoader().init(p));
     registry.put("application/javabin", new JavabinLoader(instance).init(p));
+    registry.put("application/cbor", CborStream.createLoader(p));

Review Comment:
   Looking at nearby lines of code, this suggests that perhaps CborStream should be called "CborLoader"?



##########
solr/core/src/java/org/apache/solr/response/CborResponseWriter.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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.solr.response;
+
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import com.fasterxml.jackson.dataformat.cbor.CBORGenerator;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import org.apache.solr.client.solrj.impl.BinaryResponseParser;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.request.SolrQueryRequest;
+
+public class CborResponseWriter extends BinaryResponseWriter {

Review Comment:
   javadoc on public classes



##########
solr/core/src/java/org/apache/solr/handler/loader/CborStream.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.solr.handler.loader;
+
+import com.fasterxml.jackson.core.JsonToken;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.cbor.CBORFactory;
+import com.fasterxml.jackson.dataformat.cbor.CBORParser;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Consumer;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+
+public class CborStream {

Review Comment:
   Since this appears to be an internal class; maybe comment that this is so, and mention who uses it.



-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] noblepaul merged pull request #1655: SOLR-16812: Support CBOR format for update/query

Posted by "noblepaul (via GitHub)" <gi...@apache.org>.
noblepaul merged PR #1655:
URL: https://github.com/apache/solr/pull/1655


-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] noblepaul commented on a diff in pull request #1655: SOLR-16812: Support CBOR format for update/query

Posted by "noblepaul (via GitHub)" <gi...@apache.org>.
noblepaul commented on code in PR #1655:
URL: https://github.com/apache/solr/pull/1655#discussion_r1206117294


##########
solr/solrj/src/java/org/apache/solr/client/solrj/request/GenericSolrRequest.java:
##########
@@ -42,6 +44,22 @@ public GenericSolrRequest setContentWriter(ContentWriter contentWriter) {
     return this;
   }
 
+  public GenericSolrRequest withContent(byte[] buf, String type) {

Review Comment:
   I don't think so.
   Users can already do so with their own `ContentWriter` impl. This is just a helper method
   



-- 
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: issues-unsubscribe@solr.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org