You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by gu...@apache.org on 2019/03/12 14:47:26 UTC

[lucene-solr] branch master updated: SOLR-12891 MacroExpander will no longer will expand URL parameters by default inside of the 'expr' parameter, add InjectionDefense class for safer handling of untrusted data in streaming expressions and add -DStreamingExpressionMacros system property to revert to legacy behavior

This is an automated email from the ASF dual-hosted git repository.

gus pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 9edc557  SOLR-12891 MacroExpander will no longer will expand URL parameters by default inside of the 'expr' parameter, add InjectionDefense class for safer handling of untrusted data in streaming expressions and add -DStreamingExpressionMacros system property to revert to legacy behavior
9edc557 is described below

commit 9edc557f4526ffbbf35daea06972eb2c595e692b
Author: Gus Heck <gu...@apache.org>
AuthorDate: Tue Mar 12 10:46:30 2019 -0400

    SOLR-12891 MacroExpander will no longer will expand URL parameters by
    default inside of the 'expr' parameter, add InjectionDefense class
    for safer handling of untrusted data in streaming expressions and add
    -DStreamingExpressionMacros system property to revert to legacy behavior
---
 solr/CHANGES.txt                                   |  35 ++--
 .../apache/solr/request/macro/MacroExpander.java   |  13 +-
 .../solr/request/macro/TestMacroExpander.java      |  25 +++
 solr/solr-ref-guide/src/streaming-expressions.adoc |  12 +-
 .../java/org/apache/solr/client/solrj/io/Lang.java |   2 +-
 .../solr/client/solrj/io/stream/NoOpStream.java    | 107 +++++++++++
 .../stream/expr/InjectedExpressionException.java   |  24 +++
 .../solrj/io/stream/expr/InjectionDefense.java     | 199 +++++++++++++++++++++
 .../io/stream/expr/StreamExpressionParser.java     | 116 ++++++------
 .../org/apache/solr/client/solrj/io/TestLang.java  |   2 +-
 .../solrj/io/stream/StreamExpressionTest.java      | 109 +++++------
 .../solrj/io/stream/expr/InjectionDefenseTest.java | 115 ++++++++++++
 12 files changed, 622 insertions(+), 137 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f847cda..5afd706 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -35,6 +35,11 @@ Upgrade Notes
   in the event of some hard crash. Switch back to synchronous logging if this is unacceptable, see
   see commeints in the log4j2 configuration files (log4j2.xml by default).
 
+* SOLR-12891: MacroExpander will no longer will expand URL parameters inside of the 'expr' parameter (used by streaming
+  expressions) Additionally, users are advised to use the 'InjectionDefense' class when constructing streaming
+  expressions that include user supplied data to avoid risks similar to SQL injection. The legacy behavior of
+  expanding the 'expr' parameter can be reinstated with -DStreamingExpressionMacros=true passed to the JVM at startup.
+
 New Features
 ----------------------
 
@@ -54,7 +59,7 @@ Upgrade Notes
 ----------------------
 When requesting the status of an async request via REQUESTSTATUS collections API, the response will
 include the list of internal async requests (if any) in the "success" or "failed" keys (in addition
-to them being included outside those keys for backwards compatibility). See SOLR-12708 for more 
+to them being included outside those keys for backwards compatibility). See SOLR-12708 for more
 details
 
 New Features
@@ -89,7 +94,7 @@ Bug Fixes
 
 * SOLR-12708: Async collection actions should not hide internal failures (Mano Kovacs, Varun Thacker, Tomás Fernández Löbbe)
 
-* SOLR-11883: 500 code on functional query syntax errors and parameter dereferencing errors 
+* SOLR-11883: 500 code on functional query syntax errors and parameter dereferencing errors
 (Munendra S N via Mikhail Khludnev)
 
 * SOLR-13234: Prometheus Metric Exporter not threadsafe. This changes the prometheus exporter to collect metrics
@@ -98,7 +103,7 @@ Bug Fixes
   all the cores.
   (Danyal Prout via shalin)
 
-* SOLR-9882: 500 error code on breaching timeAllowed by core and distributed (fsv) search, 
+* SOLR-9882: 500 error code on breaching timeAllowed by core and distributed (fsv) search,
   old and json facets (Mikhail Khludnev)
 
 * SOLR-13285: Updates with enum fields and javabin cause ClassCastException (noble)
@@ -108,7 +113,7 @@ Bug Fixes
 * SOLR-13254: Correct message that is logged in solrj's ConnectionManager when an exception
   occurred while reconnecting to ZooKeeper. (hu xiaodong via Christine Poerschke)
 
-* SOLR-13284: NullPointerException with 500 http status on omitted or wrong wt param. 
+* SOLR-13284: NullPointerException with 500 http status on omitted or wrong wt param.
   It's fixed by fallback to json (Munendra S N via Mikhail Khludnev)
 
 Improvements
@@ -176,7 +181,7 @@ Upgrade Notes
   it will send and can only be able to handle HTTP/1.1 requests. In case of using SSL Java 9 or latter versions are recommended.
 
 * Custom AuthenticationPlugin must provide its own setup for Http2SolrClient through
-  implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated.    
+  implementing HttpClientBuilderPlugin.setup, if not internal requests can't be authenticated.
 
 * LUCENE-7996: The 'func' query parser now returns scores that are equal to 0
   when a negative value is produced. This change is due to the fact that
@@ -201,7 +206,7 @@ Upgrade Notes
 
 * SOLR-12593: The "extraction" contrib (Solr Cell) no longer does any date parsing, and thus no longer has the
   "date.formats" configuration.  To ensure date strings are properly parsed, use ParseDateFieldUpdateProcessorFactory
-  (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein) 
+  (an URP) commonly registered with the name "parse-date" in "schemaless mode".  (David Smiley, Bar Rotstein)
 
 * SOLR-12643: Since Http2SolrClient does not support exposing connections related metrics. These metrics are no longer
   available 'QUERY.httpShardHandler.{availableConnections, leasedConnections, maxConnections, pendingConnections}',
@@ -219,7 +224,7 @@ Upgrade Notes
 
 * If you explicitly use BM25SimilarityFactory in your schema, the absolute scoring will be lower due to SOLR-13025.
   But ordering of documents will not change in the normal case. Use LegacyBM25SimilarityFactory if you need to force
-  the old 6.x/7.x scoring. Note that if you have not specified any similarity in schema or use the default 
+  the old 6.x/7.x scoring. Note that if you have not specified any similarity in schema or use the default
   SchemaSimilarityFactory, then LegacyBM25Similarity is automatically selected for 'luceneMatchVersion' < 8.0.0.
   See also explanation in Reference Guide chapter "Other Schema Elements".
 
@@ -234,7 +239,7 @@ Upgrade Notes
   This choice used to be toggleable with an internal/expert "anonChildDocs" parameter flag which is now gone.
   (David Smiley)
 
-* SOLR-11774: In 'langid' contrib, the LanguageIdentifierUpdateProcessor base class changed some method signatures. 
+* SOLR-11774: In 'langid' contrib, the LanguageIdentifierUpdateProcessor base class changed some method signatures.
   If you have a custom language identifier implementation you will need to adapt your code.
 
 * SOLR-9515: Hadoop dependencies have been upgraded to Hadoop 3.2.0 from 2.7.2. (Mark Miller, Kevin Risden)
@@ -278,7 +283,7 @@ New Features
 
 * SOLR-12799: Allow Authentication Plugins to intercept internode requests on a per-request basis.
   The BasicAuth plugin now supports a new parameter 'forwardCredentials', and when set to 'true',
-  user's BasicAuth credentials will be used instead of PKI for client initiated internode requests. (janhoy, noble) 
+  user's BasicAuth credentials will be used instead of PKI for client initiated internode requests. (janhoy, noble)
 
 * SOLR-12791: Add Metrics reporting for AuthenticationPlugin (janhoy)
 
@@ -309,7 +314,7 @@ Bug Fixes
 
 * SOLR-13058: Fix block that was synchronizing on the wrong collection in OverseerTaskProcessor (Gus Heck)
 
-* SOLR-11774: langid.map.individual now works together with langid.map.keepOrig. Also the detectLanguage() API 
+* SOLR-11774: langid.map.individual now works together with langid.map.keepOrig. Also the detectLanguage() API
   is changed to accept a Reader allowing for more memory efficient implementations (janhoy)
 
 * SOLR-13126: Query boosts were not being combined correctly for documents where not all boost queries
@@ -350,7 +355,7 @@ Optimizations
 * SOLR-12725: ParseDateFieldUpdateProcessorFactory should reuse ParsePosition. (ab)
 
 * SOLR-13025: Due to LUCENE-8563, the BM25Similarity formula no longer includes the (k1+1) factor in the numerator
-  This gives a lower absolute score but doesn't affect ordering, as this is a constant factor which is the same 
+  This gives a lower absolute score but doesn't affect ordering, as this is a constant factor which is the same
   for every document. Use LegacyBM25SimilarityFactory if you need the old 6.x/7.x scoring. See also upgrade notes (janhoy)
 
 * SOLR-13130: during the ResponseBuilder.STAGE_GET_FIELDS directly copy string bytes and avoid creating String Objects (noble)
@@ -455,16 +460,16 @@ Upgrade Notes
 New Features
 ----------------------
 
-* SOLR-12839: JSON 'terms' Faceting now supports a 'prelim_sort' option to use when initially selecting 
+* SOLR-12839: JSON 'terms' Faceting now supports a 'prelim_sort' option to use when initially selecting
   the top ranking buckets, prior to the final 'sort' option used after refinement.  (hossman)
 
 * SOLR-7896: Add a login page to Admin UI, with initial support for Basic Auth (janhoy)
 
-* SOLR-13116: Add Admin UI login support for Kerberos (janhoy, Jason Gerlowski) 
+* SOLR-13116: Add Admin UI login support for Kerberos (janhoy, Jason Gerlowski)
 
 * SOLR-11126: New Node-level health check handler at /admin/info/healthcheck and /node/health paths that
   checks if the node is live, connected to zookeeper and not shutdown. (Anshum Gupta, Amrit Sarkar, shalin)
-  
+
 * SOLR-12770: Make it possible to configure a host whitelist for distributed search
   (Christine Poerschke, janhoy, Erick Erickson, Tomás Fernández Löbbe)
 
@@ -483,7 +488,7 @@ Bug Fixes
 
 * SOLR-12546: CVSResponseWriter omits useDocValuesAsStored=true field when fl=*
   (Munendra S N via Mikhail Khludnev)
-  
+
 * SOLR-12933: Fix SolrCloud distributed commit. (Mark Miller)
 
 * SOLR-13014: URI Too Long with large streaming expressions in SolrJ (janhoy)
diff --git a/solr/core/src/java/org/apache/solr/request/macro/MacroExpander.java b/solr/core/src/java/org/apache/solr/request/macro/MacroExpander.java
index 9d432fa..67219e5 100644
--- a/solr/core/src/java/org/apache/solr/request/macro/MacroExpander.java
+++ b/solr/core/src/java/org/apache/solr/request/macro/MacroExpander.java
@@ -35,8 +35,7 @@ public class MacroExpander {
   private char escape = '\\';
   private int level;
   private final boolean failOnMissingParams;
-
-
+  
   public MacroExpander(Map<String,String[]> orig) {
     this(orig, false);
   }
@@ -58,8 +57,12 @@ public class MacroExpander {
     boolean changed = false;
     for (Map.Entry<String,String[]> entry : orig.entrySet()) {
       String k = entry.getKey();
-      String newK = expand(k);
       String[] values = entry.getValue();
+      if (!isExpandingExpr() && "expr".equals(k) ) {  // SOLR-12891
+        expanded.put(k,values);
+        continue;
+      }
+      String newK = expand(k);
       List<String> newValues = null;
       for (String v : values) {
         String newV = expand(v);
@@ -92,6 +95,10 @@ public class MacroExpander {
     return changed;
   }
 
+  private Boolean isExpandingExpr() {
+    return Boolean.valueOf(System.getProperty("StreamingExpressionMacros", "false"));
+  }
+
   public String expand(String val) {
     level++;
     try {
diff --git a/solr/core/src/test/org/apache/solr/request/macro/TestMacroExpander.java b/solr/core/src/test/org/apache/solr/request/macro/TestMacroExpander.java
index 8e3d273..733b960 100644
--- a/solr/core/src/test/org/apache/solr/request/macro/TestMacroExpander.java
+++ b/solr/core/src/test/org/apache/solr/request/macro/TestMacroExpander.java
@@ -118,6 +118,7 @@ public class TestMacroExpander extends SolrTestCase {
   public void testMap() { // see SOLR-9740, the second fq param was being dropped.
     final Map<String,String[]> request = new HashMap<>();
     request.put("fq", new String[] {"zero", "${one_ref}", "two", "${three_ref}"});
+    request.put("expr", new String[] {"${one_ref}"}); // expr is for streaming expressions, no replacement by default
     request.put("one_ref",new String[] {"one"});
     request.put("three_ref",new String[] {"three"});
     Map expanded = MacroExpander.expand(request);
@@ -125,6 +126,30 @@ public class TestMacroExpander extends SolrTestCase {
     assertEquals("one", ((String[])expanded.get("fq"))[1]);
     assertEquals("two", ((String[]) expanded.get("fq"))[2]);
     assertEquals("three", ((String[]) expanded.get("fq"))[3]);
+
+    assertEquals("${one_ref}", ((String[])expanded.get("expr"))[0]);
   }
 
+  @Test
+  public void testMapExprExpandOn() {
+    final Map<String,String[]> request = new HashMap<>();
+    request.put("fq", new String[] {"zero", "${one_ref}", "two", "${three_ref}"});
+    request.put("expr", new String[] {"${one_ref}"}); // expr is for streaming expressions, no replacement by default
+    request.put("one_ref",new String[] {"one"});
+    request.put("three_ref",new String[] {"three"});
+    // I believe that so long as this is sure to be reset before the end of the test we should
+    // be fine with respect to other tests.
+    String oldVal = System.getProperty("StreamingExpressionMacros","false");
+    System.setProperty("StreamingExpressionMacros", "true");
+    try {
+      Map expanded = MacroExpander.expand(request);
+      assertEquals("zero", ((String[])expanded.get("fq"))[0]);
+      assertEquals("one", ((String[])expanded.get("fq"))[1]);
+      assertEquals("two", ((String[]) expanded.get("fq"))[2]);
+      assertEquals("three", ((String[]) expanded.get("fq"))[3]);
+      assertEquals("one", ((String[])expanded.get("expr"))[0]);
+    } finally {
+      System.setProperty("StreamingExpressionMacros", oldVal);
+    }
+  }
 }
diff --git a/solr/solr-ref-guide/src/streaming-expressions.adoc b/solr/solr-ref-guide/src/streaming-expressions.adoc
index 97eb297..c1c0404 100644
--- a/solr/solr-ref-guide/src/streaming-expressions.adoc
+++ b/solr/solr-ref-guide/src/streaming-expressions.adoc
@@ -99,11 +99,17 @@ The {solr-javadocs}/solr-solrj/org/apache/solr/client/solrj/io/package-summary.h
 
 [source,java]
 ----
-StreamFactory streamFactory = new DefaultStreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress());
-
-ParallelStream pstream = (ParallelStream)streamFactory.constructStream("parallel(collection1, group(search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc,a_f asc\", partitionKeys=\"a_s\"), by=\"a_s asc\"), workers=\"2\", zkHost=\""+zkHost+"\", sort=\"a_s asc\")");
+    StreamFactory streamFactory = new DefaultStreamFactory().withCollectionZkHost("collection1", zkServer.getZkAddress());
+    InjectionDefense defense = new InjectionDefense("parallel(collection1, group(search(collection1, q=\"*:*\", fl=\"id,a_s,a_i,a_f\", sort=\"a_s asc,a_f asc\", partitionKeys=\"a_s\"), by=\"a_s asc\"), workers=\"2\", zkHost=\"?$?\", sort=\"a_s asc\")");
+    defense.addParameter(zkhost);
+    ParallelStream pstream = (ParallelStream)streamFactory.constructStream(defense.safeExpressionString());
 ----
 
+Note that InjectionDefense need only be used if the string being inserted could contain user supplied data. See the
+javadoc for `InjectionDefense` for usage details and SOLR-12891 for an example of the potential risks.
+Also note that for security reasons normal parameter substitution no longer applies to the expr parameter
+unless the jvm has been started with `-DStreamingExpressionMacros=true` (usually via `solr.in.sh`)
+
 === Data Requirements
 
 Because streaming expressions relies on the `/export` handler, many of the field and field type requirements to use `/export` are also requirements for `/stream`, particularly for `sort` and `fl` parameters. Please see the section <<exporting-result-sets.adoc#exporting-result-sets,Exporting Result Sets>> for details.
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
index 0e3d2b8..ab14062 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -94,7 +94,7 @@ public class Lang {
         .withFunctionName("plist", ParallelListStream.class)
         .withFunctionName("zplot", ZplotStream.class)
         .withFunctionName("hashRollup", HashRollupStream.class)
-
+        .withFunctionName("noop", NoOpStream.class)
 
         // metrics
         .withFunctionName("min", MinMetric.class)
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NoOpStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NoOpStream.java
new file mode 100644
index 0000000..8d55c31
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/NoOpStream.java
@@ -0,0 +1,107 @@
+/*
+ * 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.client.solrj.io.stream;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.comp.StreamComparator;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation;
+import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExplanation;
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+/**
+ * A simple no-operation stream. Immediately returns eof. Mostly intended for use as
+ * a place holder in {@link org.apache.solr.client.solrj.io.stream.expr.InjectionDefense}.
+ *
+ * @since 8.0.0
+ */
+public class NoOpStream extends TupleStream implements Expressible {
+
+  private static final long serialVersionUID = 1;
+  private boolean finished;
+
+
+
+  public NoOpStream() throws IOException {
+  }
+
+  public NoOpStream(StreamExpression expression, StreamFactory factory) throws IOException {
+  }
+
+
+  @Override
+  public StreamExpression toExpression(StreamFactory factory) throws IOException{
+    return toExpression(factory, true);
+  }
+
+  private StreamExpression toExpression(StreamFactory factory, boolean includeStreams) throws IOException {
+    // function name
+    StreamExpression expression = new StreamExpression(factory.getFunctionName(this.getClass()));
+
+    return expression;
+  }
+
+  @Override
+  public Explanation toExplanation(StreamFactory factory) throws IOException {
+
+    return new StreamExplanation(getStreamNodeId().toString())
+        .withFunctionName(factory.getFunctionName(this.getClass()))
+        .withImplementingClass(this.getClass().getName())
+        .withExpressionType(ExpressionType.STREAM_DECORATOR)
+        .withExpression(toExpression(factory, false).toString());
+  }
+
+  public void setStreamContext(StreamContext context) {
+  }
+
+  public List<TupleStream> children() {
+    List<TupleStream> l =  new ArrayList<TupleStream>();
+    return l;
+  }
+
+  public void open() throws IOException {
+
+  }
+
+  public void close() throws IOException {
+  }
+
+  public Tuple read() throws IOException {
+      HashMap m = new HashMap();
+      m.put("EOF", true);
+      Tuple tuple = new Tuple(m);
+      return tuple;
+  }
+
+  /** Return the stream sort - ie, the order in which records are returned */
+  public StreamComparator getStreamSort(){
+    return null;
+  }
+
+  public int getCost() {
+    return 0;
+  }
+
+
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectedExpressionException.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectedExpressionException.java
new file mode 100644
index 0000000..82d05f3
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectedExpressionException.java
@@ -0,0 +1,24 @@
+/*
+ * 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.client.solrj.io.stream.expr;
+
+class InjectedExpressionException extends IllegalStateException {
+  InjectedExpressionException(String s) {
+    super(s);
+  }
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectionDefense.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectionDefense.java
new file mode 100644
index 0000000..69c43c7
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/InjectionDefense.java
@@ -0,0 +1,199 @@
+/*
+ * 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.client.solrj.io.stream.expr;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * A class with which to safely build a streaming expression. Three types of parameters
+ * (String, Numeric, Expression) are accepted and minimally type checked. All parameters
+ * are positional (unnamed) so the order in which parameters are added must correspond to
+ * the order of the parameters in the supplied expression string.<br><br>
+ *
+ * <p>Specifically, this class verifies that the parameter substitutions do not inject
+ * additional expressions, and that the parameters are strings, valid numbers or valid
+ * expressions producing the expected number of sub-expressions. The idea is not to provide
+ * full type safety but rather to heuristically prevent the injection of malicious
+ * expressions. The template expression and the parameters supplied must not contain
+ * comments since injection of  comments could be used to hide one or more of the expected
+ * expressions. Use {@link #stripComments(String)} to remove comments.<br><br>
+ *
+ * <p>Valid patterns for parameters are:
+ * <ul>
+ * <li>?$? for strings</li>
+ * <li>?#? for numeric parameters in integer or decimal format (no exponents)</li>
+ * <li>?(n)? for expressions producing n sub-expressions (minimum n=1)</li>
+ * </ul>
+ *
+ * @since 8.0.0
+ */
+
+public class InjectionDefense {
+
+  private static final Pattern STRING_PARAM = Pattern.compile("\\?\\$\\?");
+  private static final Pattern NUMBER_PARAM = Pattern.compile("\\?#\\?");
+  private static final Pattern EXPRESSION_PARAM = Pattern.compile("\\?\\(\\d+\\)\\?");
+  private static final Pattern EXPRESSION_COUNT = Pattern.compile("\\d+");
+  private static final Pattern ANY_PARAM = Pattern.compile("\\?(?:[$#]|(?:\\(\\d+\\)))\\?");
+  private static final Pattern INT_OR_FLOAT = Pattern.compile("-?\\d+(?:\\.\\d+)?");
+
+  private String exprString;
+  private int expressionCount;
+  private List<String> params = new ArrayList<>();
+
+  @SuppressWarnings("WeakerAccess")
+  public InjectionDefense(String exprString) {
+    this.exprString = exprString;
+    checkExpression(exprString);
+  }
+
+  @SuppressWarnings("WeakerAccess")
+  public static String stripComments(String exprString) {
+    return StreamExpressionParser.stripComments(exprString);
+  }
+
+  public void addParameter(String param) {
+    params.add(param);
+  }
+
+  /**
+   * Provides an expression that is guaranteed to have the expected number of sub-expressions
+   *
+   * @return An expression object that should be safe from injection of additional expressions
+   */
+  @SuppressWarnings("WeakerAccess")
+  public StreamExpression safeExpression() {
+    String exprStr = buildExpression();
+    System.out.println(exprStr);
+    StreamExpression parsed = StreamExpressionParser.parse(exprStr);
+    int actual = countExpressions(parsed);
+    if (actual != expressionCount) {
+      throw new InjectedExpressionException("Expected Expression count ("+expressionCount+") does not match actual final " +
+          "expression count ("+actual+")! (possible injection attack?)");
+    } else {
+      return parsed;
+    }
+  }
+
+  /**
+   * Provides a string that is guaranteed to parse to a legal expression and to have the expected
+   * number of sub-expressions.
+   *
+   * @return A string that should be safe from injection of additional expressions.
+   */
+  @SuppressWarnings("WeakerAccess")
+  public String safeExpressionString() {
+    String exprStr = buildExpression();
+    StreamExpression parsed = StreamExpressionParser.parse(exprStr);
+    if (countExpressions(parsed) != expressionCount) {
+      throw new InjectedExpressionException("Expected Expression count does not match Actual final " +
+          "expression count! (possible injection attack?)");
+    } else {
+      return exprStr;
+    }
+
+  }
+
+  String buildExpression() {
+    Matcher anyParam = ANY_PARAM.matcher(exprString);
+    StringBuffer buff = new StringBuffer();
+    int pIdx = 0;
+    while (anyParam.find()) {
+      String found = anyParam.group();
+      String p = params.get(pIdx++);
+      if (found.contains("#")) {
+        if (!INT_OR_FLOAT.matcher(p).matches()) {
+          throw new NumberFormatException("Argument " + pIdx + " (" + p + ") is not numeric!");
+        }
+      }
+      anyParam.appendReplacement(buff, p);
+    }
+    anyParam.appendTail(buff);
+
+    // strip comments may add '\n' at the end so trim()
+    String result = buff.toString().trim();
+    String noComments = stripComments(result).trim();
+    if (!result.equals(noComments)) {
+      throw new IllegalStateException("Comments are not allowed in prepared expressions for security reasons " +
+          "please pre-process stripComments() first. If there were no comments, then they have been injected by " +
+          "a parameter value.");
+    }
+    return buff.toString().trim();
+  }
+
+  /**
+   * Perform some initial checks and establish the expected number of expressions
+   *
+   * @param exprString the expression to check.
+   */
+  private void checkExpression(String exprString) {
+    exprString = STRING_PARAM.matcher(exprString).replaceAll("foo");
+    exprString = NUMBER_PARAM.matcher(exprString).replaceAll("0");
+    Matcher eMatcher = EXPRESSION_PARAM.matcher(exprString);
+    StringBuffer temp = new StringBuffer();
+    while (eMatcher.find()) {
+      Matcher counter = EXPRESSION_COUNT.matcher(eMatcher.group());
+      eMatcher.appendReplacement(temp, "noop()");
+      if (counter.find()) {
+        Integer subExprCount = Integer.valueOf(counter.group());
+        if (subExprCount < 1) {
+          throw new IllegalStateException("Expression Param must contribute at least 1 expression!" +
+              " ?(1)? is the minimum allowed ");
+        }
+        expressionCount += (subExprCount - 1); // the noop() we insert will get counted later.
+      }
+    }
+    eMatcher.appendTail(temp);
+    exprString = temp.toString();
+
+    StreamExpression parsed = StreamExpressionParser.parse(exprString);
+    if (parsed != null) {
+      expressionCount += countExpressions(parsed);
+    } else {
+      throw new IllegalStateException("Invalid expression (parse returned null):" + exprString);
+    }
+  }
+
+  private int countExpressions(StreamExpression expression) {
+    int result = 0;
+    List<StreamExpressionParameter> exprToCheck = new ArrayList<>();
+    exprToCheck.add(expression);
+    while (exprToCheck.size() > 0) {
+      StreamExpressionParameter remove = exprToCheck.remove(0);
+      if (remove instanceof StreamExpressionNamedParameter) {
+        remove = ((StreamExpressionNamedParameter) remove).getParameter();
+      }
+      if (remove instanceof StreamExpression) {
+        result++;
+        for (StreamExpressionParameter parameter : ((StreamExpression) remove).getParameters()) {
+          if (parameter instanceof StreamExpressionNamedParameter) {
+            parameter = ((StreamExpressionNamedParameter) parameter).getParameter();
+          }
+          if (parameter instanceof StreamExpression) {
+            exprToCheck.add(parameter);
+          }
+        }
+      }
+    }
+    return result;
+  }
+
+}
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamExpressionParser.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamExpressionParser.java
index 7ae797f..45d5a80 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamExpressionParser.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/expr/StreamExpressionParser.java
@@ -41,43 +41,35 @@ public class StreamExpressionParser {
     if(null != expr && expr instanceof StreamExpression){
       return (StreamExpression)expr;
     }
-    
+
     return null;
   }
 
 
-  private static String stripComments(String clause) throws RuntimeException {
+  static String stripComments(String clause) throws RuntimeException {
     StringBuilder builder = new StringBuilder();
-    BufferedReader reader = null;
 
-    try {
-      reader = new BufferedReader(new StringReader(clause));
-      String line = null;
+    try (BufferedReader reader = new BufferedReader(new StringReader(clause))) {
+      String line;
       while ((line = reader.readLine()) != null) {
-        if(line.trim().startsWith("#")) {
-          continue;
-        } else {
-          builder.append(line+'\n');
+        if (!line.trim().startsWith("#")) {
+          builder.append(line).append('\n');
         }
       }
-    }catch(Exception e) {
+    } catch (Exception e) {
       throw new RuntimeException(e);
-    } finally{
-      try {
-        reader.close();
-      } catch (Exception e) {}
     }
 
     return builder.toString();
   }
-  
-  private static StreamExpressionParameter generateStreamExpression(String clause){    
+
+  private static StreamExpressionParameter generateStreamExpression(String clause){
     String working = clause.trim();
-    
+
     if(!isExpressionClause(working)){
       throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper expression clause", working));
     }
-    
+
     // Get functionName
     int firstOpenParen = findNextClear(working, 0, '(');
     StreamExpression expression = new StreamExpression(working.substring(0, firstOpenParen).trim());
@@ -85,7 +77,7 @@ public class StreamExpressionParser {
     // strip off functionName and ()
     working = working.substring(firstOpenParen + 1,working.length() - 1).trim();
     List<String> parts = splitOn(working,',');
-        
+
     for(int idx = 0; idx < parts.size(); ++idx){
       String part = parts.get(idx).trim();
       if(isExpressionClause(part)){
@@ -104,18 +96,18 @@ public class StreamExpressionParser {
         expression.addParameter(new StreamExpressionValue(part));
       }
     }
-    
+
     return expression;
   }
 
-  private static StreamExpressionNamedParameter generateNamedParameterExpression(String clause){    
+  private static StreamExpressionNamedParameter generateNamedParameterExpression(String clause){
     String working = clause.trim();
-    
+
     // might be overkill as the only place this is called from does this check already
     if(!isNamedParameterClause(working)){
       throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper named parameter clause", working));
     }
-    
+
     // Get name
     int firstOpenEquals = findNextClear(working, 0, '=');
     StreamExpressionNamedParameter namedParameter = new StreamExpressionNamedParameter(working.substring(0, firstOpenEquals).trim());
@@ -133,7 +125,7 @@ public class StreamExpressionParser {
           throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper named parameter clause", working));
         }
       }
-      
+
       // if contain \" replace with "
       if(parameter.contains("\\\"")){
         parameter = parameter.replace("\\\"", "\"");
@@ -141,46 +133,46 @@ public class StreamExpressionParser {
           throw new IllegalArgumentException(String.format(Locale.ROOT,"'%s' is not a proper named parameter clause", working));
         }
       }
-      
+
       namedParameter.setParameter(new StreamExpressionValue(parameter));
     }
-    
+
     return namedParameter;
   }
 
-  
+
   /* Returns true if the clause is a valid expression clause. This is defined to
    * mean it begins with ( and ends with )
    * Expects that the passed in clause has already been trimmed of leading and
    * trailing spaces*/
   private static boolean isExpressionClause(String clause){
     // operator(.....something.....)
-    
+
     // must be balanced
     if(!isBalanced(clause)){ return false; }
-    
+
     // find first (, then check from start to that location and only accept alphanumeric
     int firstOpenParen = findNextClear(clause, 0, '(');
     if(firstOpenParen <= 0 || firstOpenParen == clause.length() - 1){ return false; }
     String functionName = clause.substring(0, firstOpenParen).trim();
     if(!wordToken(functionName)){ return false; }
-    
+
     // Must end with )
     return clause.endsWith(")");
   }
-  
+
   private static boolean isNamedParameterClause(String clause){
     // name=thing
-    
+
     // find first = then check from start to that location and only accept alphanumeric
     int firstOpenEquals = findNextClear(clause, 0, '=');
     if(firstOpenEquals <= 0 || firstOpenEquals == clause.length() - 1){ return false; }
     String name = clause.substring(0, firstOpenEquals);
     if(!wordToken(name.trim())){ return false; }
-    
+
     return true;
   }
-  
+
   /* Finds index of the next char equal to findThis that is not within a quote or set of parens
    * Does not work with the following values of findThis: " ' \ ) -- well, it might but wouldn't
    * really give you what you want. Don't call with those characters */
@@ -189,22 +181,22 @@ public class StreamExpressionParser {
     boolean isDoubleQuote = false;
     boolean isSingleQuote = false;
     boolean isEscaped = false;
-    
+
     for(int idx = startingIdx; idx < clause.length(); ++idx){
       char c = clause.charAt(idx);
-      
+
       // if we're not in a non-escaped quote or paren state, then we've found the space we want
       if(c == findThis && !isEscaped && !isSingleQuote && !isDoubleQuote && 0 == openParens){
         return idx;
       }
-      
-      
+
+
       switch(c){
         case '\\':
           // We invert to support situations where \\ exists
           isEscaped = !isEscaped;
           break;
-          
+
         case '"':
           // if we're not in a non-escaped single quote state, then invert the double quote state
           if(!isEscaped && !isSingleQuote){
@@ -212,7 +204,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-        
+
         case '\'':
           // if we're not in a non-escaped double quote state, then invert the single quote state
           if(!isEscaped && !isDoubleQuote){
@@ -220,7 +212,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-          
+
         case '(':
           // if we're not in a non-escaped quote state, then increment the # of open parens
           if(!isEscaped && !isSingleQuote && !isDoubleQuote){
@@ -228,7 +220,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-          
+
         case ')':
           // if we're not in a non-escaped quote state, then decrement the # of open parens
           if(!isEscaped && !isSingleQuote && !isDoubleQuote){
@@ -242,10 +234,10 @@ public class StreamExpressionParser {
     }
 
     // Not found
-    return -1;    
+    return -1;
   }
 
-  
+
   /* Returns a list of the tokens found. Assumed to be of the form
    * 'foo bar baz' and not of the for '(foo bar baz)'
    * 'foo bar (baz jaz)' is ok and will return three tokens of
@@ -253,46 +245,46 @@ public class StreamExpressionParser {
    */
   private static List<String> splitOn(String clause, char splitOnThis){
     String working = clause.trim();
-    
+
     List<String> parts = new ArrayList<String>();
-    
+
     while(true){ // will break when next splitOnThis isn't found
       int nextIdx = findNextClear(working, 0, splitOnThis);
-      
+
       if(nextIdx < 0){
         parts.add(working);
         break;
       }
-      
+
       parts.add(working.substring(0, nextIdx));
-      
+
       // handle ending splitOnThis
       if(nextIdx+1 == working.length()){
         break;
       }
-      
-      working = working.substring(nextIdx + 1).trim();      
+
+      working = working.substring(nextIdx + 1).trim();
     }
-    
+
     return parts;
   }
-  
+
   /* Returns true if the clause has balanced parenthesis */
   private static boolean isBalanced(String clause){
     int openParens = 0;
     boolean isDoubleQuote = false;
     boolean isSingleQuote = false;
     boolean isEscaped = false;
-    
+
     for(int idx = 0; idx < clause.length(); ++idx){
       char c = clause.charAt(idx);
-      
+
       switch(c){
         case '\\':
           // We invert to support situations where \\ exists
           isEscaped = !isEscaped;
           break;
-          
+
         case '"':
           // if we're not in a non-escaped single quote state, then invert the double quote state
           if(!isEscaped && !isSingleQuote){
@@ -300,7 +292,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-        
+
         case '\'':
           // if we're not in a non-escaped double quote state, then invert the single quote state
           if(!isEscaped && !isDoubleQuote){
@@ -308,7 +300,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-          
+
         case '(':
           // if we're not in a non-escaped quote state, then increment the # of open parens
           if(!isEscaped && !isSingleQuote && !isDoubleQuote){
@@ -316,12 +308,12 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-          
+
         case ')':
           // if we're not in a non-escaped quote state, then decrement the # of open parens
           if(!isEscaped && !isSingleQuote && !isDoubleQuote){
             openParens -= 1;
-            
+
             // If we're ever < 0 then we know we're not balanced
             if(openParens < 0){
               return false;
@@ -329,7 +321,7 @@ public class StreamExpressionParser {
           }
           isEscaped = false;
           break;
-          
+
         default:
           isEscaped = false;
       }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
index da155fb..b43e963 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -75,7 +75,7 @@ public class TestLang extends SolrTestCase {
       "convexHull", "getVertices", "getBaryCenter", "getArea", "getBoundarySize","oscillate",
       "getAmplitude", "getPhase", "getAngularFrequency", "enclosingDisk", "getCenter", "getRadius",
       "getSupportPoints", "pairSort", "log10", "plist", "recip", "pivot", "ltrim", "rtrim", "export",
-      "zplot", "natural", "repeat", "movingMAD", "hashRollup"};
+      "zplot", "natural", "repeat", "movingMAD", "hashRollup", "noop"};
 
   @Test
   public void testLang() {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
index e271401..d4b6ce2 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamExpressionTest.java
@@ -412,51 +412,56 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
   @Test
   public void testParameterSubstitution() throws Exception {
+    String oldVal = System.getProperty("StreamingExpressionMacros", "false");
+    System.setProperty("StreamingExpressionMacros", "true");
+    try {
+      new UpdateRequest()
+          .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
+          .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
+          .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
+          .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
+          .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
+          .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
 
-    new UpdateRequest()
-        .add(id, "0", "a_s", "hello0", "a_i", "0", "a_f", "0")
-        .add(id, "2", "a_s", "hello2", "a_i", "2", "a_f", "0")
-        .add(id, "3", "a_s", "hello3", "a_i", "3", "a_f", "3")
-        .add(id, "4", "a_s", "hello4", "a_i", "4", "a_f", "4")
-        .add(id, "1", "a_s", "hello1", "a_i", "1", "a_f", "1")
-        .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
-
-    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/" + COLLECTIONORALIAS;
-    List<Tuple> tuples;
-    TupleStream stream;
+      String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString() + "/" + COLLECTIONORALIAS;
+      List<Tuple> tuples;
+      TupleStream stream;
 
-    // Basic test
-    ModifiableSolrParams sParams = new ModifiableSolrParams();
-    sParams.set("expr", "merge("
-        + "${q1},"
-        + "${q2},"
-        + "on=${mySort})");
-    sParams.set(CommonParams.QT, "/stream");
-    sParams.set("q1", "search(" + COLLECTIONORALIAS + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
-    sParams.set("q2", "search(" + COLLECTIONORALIAS + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
-    sParams.set("mySort", "a_f asc");
-    stream = new SolrStream(url, sParams);
-    tuples = getTuples(stream);
+      // Basic test
+      ModifiableSolrParams sParams = new ModifiableSolrParams();
+      sParams.set("expr", "merge("
+          + "${q1},"
+          + "${q2},"
+          + "on=${mySort})");
+      sParams.set(CommonParams.QT, "/stream");
+      sParams.set("q1", "search(" + COLLECTIONORALIAS + ", q=\"id:(0 3 4)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
+      sParams.set("q2", "search(" + COLLECTIONORALIAS + ", q=\"id:(1)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
+      sParams.set("mySort", "a_f asc");
+      stream = new SolrStream(url, sParams);
+      tuples = getTuples(stream);
 
-    assertEquals(4, tuples.size());
-    assertOrder(tuples, 0,1,3,4);
+      assertEquals(4, tuples.size());
+      assertOrder(tuples, 0, 1, 3, 4);
 
-    // Basic test desc
-    sParams.set("mySort", "a_f desc");
-    stream = new SolrStream(url, sParams);
-    tuples = getTuples(stream);
+      // Basic test desc
+      sParams.set("mySort", "a_f desc");
+      stream = new SolrStream(url, sParams);
+      tuples = getTuples(stream);
 
-    assertEquals(4, tuples.size());
-    assertOrder(tuples, 4, 3, 1, 0);
+      assertEquals(4, tuples.size());
+      assertOrder(tuples, 4, 3, 1, 0);
 
-    // Basic w/ multi comp
-    sParams.set("q2", "search(" + COLLECTIONORALIAS + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
-    sParams.set("mySort", "\"a_f asc, a_s asc\"");
-    stream = new SolrStream(url, sParams);
-    tuples = getTuples(stream);
+      // Basic w/ multi comp
+      sParams.set("q2", "search(" + COLLECTIONORALIAS + ", q=\"id:(1 2)\", fl=\"id,a_s,a_i,a_f\", sort=${mySort})");
+      sParams.set("mySort", "\"a_f asc, a_s asc\"");
+      stream = new SolrStream(url, sParams);
+      tuples = getTuples(stream);
 
-    assertEquals(5, tuples.size());
-    assertOrder(tuples, 0, 2, 1, 3, 4);
+      assertEquals(5, tuples.size());
+      assertOrder(tuples, 0, 2, 1, 3, 4);
+    } finally {
+      System.setProperty("StreamingExpressionMacros", oldVal);
+    }
   }
 
 
@@ -693,8 +698,8 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     .withFunctionName("min", MinMetric.class)
     .withFunctionName("max", MaxMetric.class)
     .withFunctionName("avg", MeanMetric.class)
-    .withFunctionName("count", CountMetric.class);     
-  
+    .withFunctionName("count", CountMetric.class);
+
     StreamExpression expression;
     TupleStream stream;
     List<Tuple> tuples;
@@ -847,11 +852,11 @@ public class StreamExpressionTest extends SolrCloudTestCase {
         .add(id, "8", "a_s", "hello3", "a_i", "13", "a_f", "9")
         .add(id, "9", "a_s", "hello0", "a_i", "14", "a_f", "10")
         .commit(cluster.getSolrClient(), COLLECTIONORALIAS);
-    
+
     String clause;
     TupleStream stream;
     List<Tuple> tuples;
-    
+
     StreamFactory factory = new StreamFactory()
       .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
       .withFunctionName("facet", FacetStream.class)
@@ -860,7 +865,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       .withFunctionName("max", MaxMetric.class)
       .withFunctionName("avg", MeanMetric.class)
       .withFunctionName("count", CountMetric.class);
-    
+
     // Basic test
     clause = "facet("
               +   "collection1, "
@@ -876,7 +881,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
               +   "avg(a_i), avg(a_f), "
               +   "count(*)"
               + ")";
-    
+
     stream = factory.constructStream(clause);
     tuples = getTuples(stream);
 
@@ -1526,7 +1531,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     String clause;
     TupleStream stream;
     List<Tuple> tuples;
-    
+
     StreamFactory factory = new StreamFactory()
       .withCollectionZkHost("collection1", cluster.getZkServer().getZkAddress())
       .withFunctionName("facet", FacetStream.class)
@@ -1535,7 +1540,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
       .withFunctionName("max", MaxMetric.class)
       .withFunctionName("avg", MeanMetric.class)
       .withFunctionName("count", CountMetric.class);
-    
+
     // Basic test
     clause = "facet("
               +   "collection1, "
@@ -1545,7 +1550,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
               +   "bucketSizeLimit=100, "
               +   "sum(a_i), count(*)"
               + ")";
-    
+
     stream = factory.constructStream(clause);
     tuples = getTuples(stream);
 
@@ -2796,7 +2801,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
     StreamContext streamContext = new StreamContext();
     streamContext.setSolrClientCache(cache);
     String longQuery = "\"id:(" + IntStream.range(0, 4000).mapToObj(i -> "a").collect(Collectors.joining(" ", "", "")) + ")\"";
-        
+
     try {
       assertSuccess("significantTerms("+COLLECTIONORALIAS+", q="+longQuery+", field=\"test_t\", limit=3, minTermLength=1, maxDocFreq=\".5\")", streamContext);
       String expr = "timeseries("+COLLECTIONORALIAS+", q="+longQuery+", start=\"2013-01-01T01:00:00.000Z\", " +
@@ -2821,7 +2826,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
                     +   "count(*)"
                     + ")";
       assertSuccess(expr, streamContext);
-      expr = "stats(" + COLLECTIONORALIAS + ", q="+longQuery+", sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))"; 
+      expr = "stats(" + COLLECTIONORALIAS + ", q="+longQuery+", sum(a_i), sum(a_f), min(a_i), min(a_f), max(a_i), max(a_f), avg(a_i), avg(a_f), count(*))";
       assertSuccess(expr, streamContext);
       expr = "search(" + COLLECTIONORALIAS + ", q="+longQuery+", fl=\"id,a_s,a_i,a_f\", sort=\"a_f asc, a_i asc\")";
       assertSuccess(expr, streamContext);
@@ -2880,10 +2885,10 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
     return true;
   }
-  
+
   public boolean assertString(Tuple tuple, String fieldName, String expected) throws Exception {
     String actual = (String)tuple.get(fieldName);
-    
+
     if( (null == expected && null != actual) ||
         (null != expected && null == actual) ||
         (null != expected && !expected.equals(actual))){
@@ -2901,7 +2906,7 @@ public class StreamExpressionTest extends SolrCloudTestCase {
 
     return true;
   }
-  
+
   protected boolean assertMaps(List<Map> maps, int... ids) throws Exception {
     if(maps.size() != ids.length) {
       throw new Exception("Expected id count != actual map count:"+ids.length+":"+maps.size());
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/expr/InjectionDefenseTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/expr/InjectionDefenseTest.java
new file mode 100644
index 0000000..4e39500
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/expr/InjectionDefenseTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.client.solrj.io.stream.expr;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class InjectionDefenseTest {
+
+  private static final String EXPLOITABLE = "let(a=search(foo,q=\"time_dt:[?$? TO ?$?]\",fl=\"id,time_dt\",sort=\"time_dt asc\"))";
+  private static final String NUMBER = "let(a=search(foo,q=\"gallons_f:[?#? TO ?#?]\",fl=\"id,gallons_f,time_dt\",sort=\"time_dt asc\"))";
+  private static final String NUMBER_OK = "let(a=search(foo,q=\"gallons_f:[2 TO 3.5]\",fl=\"id,gallons_f,time_dt\",sort=\"time_dt asc\"))";
+  private static final String ALLOWED = "let(a=search(foo,q=\"time_dt:[?$? TO ?$?]\",fl=\"id,time_dt\",sort=\"time_dt asc\"), x=?(2)?)";
+  private static final String INJECTED = "let(a=search(foo,q=\"time_dt:[2000-01-01T00:00:00Z TO 2020-01-01T00:00:00Z]\",fl=\"id,time_dt\",sort=\"time_dt asc\"), x=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\"),z=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from race_cars\",sort=\"id asc\"))";
+
+  @Test(expected = InjectedExpressionException.class)
+  public void testSafeExpression() {
+
+    InjectionDefense defender = new InjectionDefense(EXPLOITABLE);
+
+    defender.addParameter("2000-01-01T00:00:00Z");
+    defender.addParameter("2020-01-01T00:00:00Z]\",fl=\"id\",sort=\"id asc\"), b=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\"),c=search(foo,q=\"time_dt:[* TO 2020-01-01T00:00:00Z");
+
+    defender.safeExpression();
+  }
+
+  @Test(expected = InjectedExpressionException.class)
+  public void testSafeString() {
+
+    InjectionDefense defender = new InjectionDefense(EXPLOITABLE);
+
+    defender.addParameter("2000-01-01T00:00:00Z");
+    defender.addParameter("2020-01-01T00:00:00Z]\",fl=\"id\",sort=\"id asc\"), b=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\"),c=search(foo,q=\"time_dt:[* TO 2020-01-01T00:00:00Z");
+
+    defender.safeExpressionString();
+  }
+
+  @Test
+  public void testExpectedInjectionOfExpressions() {
+    InjectionDefense defender = new InjectionDefense(ALLOWED);
+
+    defender.addParameter("2000-01-01T00:00:00Z");
+    defender.addParameter("2020-01-01T00:00:00Z");
+    defender.addParameter("jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\"),z=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from race_cars\",sort=\"id asc\")");
+
+    // no exceptions
+    assertNotNull(defender.safeExpression());
+    assertEquals(INJECTED, defender.safeExpressionString());
+
+  }
+
+  @Test(expected = InjectedExpressionException.class)
+  public void testWrongNumberInjected() {
+    InjectionDefense defender = new InjectionDefense(ALLOWED);
+
+    defender.addParameter("2000-01-01T00:00:00Z");
+    defender.addParameter("2020-01-01T00:00:00Z");
+    defender.addParameter("jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\")");
+
+    // no exceptions
+    defender.safeExpression();
+    assertEquals(INJECTED, defender.safeExpressionString());
+
+  }
+
+  @Test
+  public void testBuildExpression() {
+    InjectionDefense defender = new InjectionDefense(ALLOWED);
+
+    defender.addParameter("2000-01-01T00:00:00Z");
+    defender.addParameter("2020-01-01T00:00:00Z");
+    defender.addParameter("jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from users\",sort=\"id asc\"),z=jdbc( connection=\"jdbc:postgresql://localhost:5432/ouchdb\",sql=\"select * from race_cars\",sort=\"id asc\")");
+
+    assertEquals(INJECTED, defender.buildExpression());
+  }
+
+  @Test
+  public void testInjectNumber() {
+    InjectionDefense defender = new InjectionDefense(NUMBER);
+
+    defender.addParameter("2");
+    defender.addParameter("3.5");
+
+    assertEquals(NUMBER_OK, defender.buildExpression());
+  }
+
+  @Test(expected = NumberFormatException.class)
+  public void testInjectAlphaFail() {
+    InjectionDefense defender = new InjectionDefense(NUMBER);
+
+    defender.addParameter("a");
+    defender.addParameter("3.5");
+
+    defender.buildExpression();
+
+  }
+}
+