You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2017/06/07 14:11:42 UTC

[38/50] [abbrv] lucene-solr:feature/autoscaling: SOLR-3702: concat(..) function query

SOLR-3702: concat(..) function query


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/357f4dfb
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/357f4dfb
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/357f4dfb

Branch: refs/heads/feature/autoscaling
Commit: 357f4dfb1840af86395dde878b55aff77c8d7e2d
Parents: 99093ca
Author: Mikhail Khludnev <mk...@apache.org>
Authored: Sat Jun 3 19:35:29 2017 +0300
Committer: Mikhail Khludnev <mk...@apache.org>
Committed: Mon Jun 5 23:26:06 2017 +0300

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../StatsCollectorSupplierFactory.java          |   2 +-
 .../solr/analytics/util/AnalyticsParams.java    |   3 +-
 .../util/valuesource/ConcatStringFunction.java  |  54 -------
 .../util/valuesource/MultiStringFunction.java   | 146 -------------------
 .../apache/solr/search/ValueSourceParser.java   |   9 ++
 .../search/function/ConcatStringFunction.java   |  53 +++++++
 .../search/function/MultiStringFunction.java    | 146 +++++++++++++++++++
 .../apache/solr/search/QueryEqualityTest.java   |  15 ++
 .../solr/search/function/TestFunctionQuery.java |  19 +++
 solr/solr-ref-guide/src/function-queries.adoc   |   1 +
 11 files changed, 248 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a11c470..df071eb 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -276,6 +276,8 @@ New Features
 * SOLR-10479: Adds support for HttpShardHandlerFactory.loadBalancerRequests(MinimumAbsolute|MaximumFraction)
   configuration. (Ramsey Haddad, Daniel Collins, Christine Poerschke)
 
+* SOLR-3702: concat(...) function query (Andrey Kudryavtsev via Mikhail Khludnev)
+
 Bug Fixes
 ----------------------
 * SOLR-10723 JSON Facet API: resize() implemented incorrectly for CountSlotAcc, HllAgg.NumericAcc

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/contrib/analytics/src/java/org/apache/solr/analytics/statistics/StatsCollectorSupplierFactory.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/statistics/StatsCollectorSupplierFactory.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/statistics/StatsCollectorSupplierFactory.java
index 30baec4..e22362d 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/statistics/StatsCollectorSupplierFactory.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/statistics/StatsCollectorSupplierFactory.java
@@ -36,7 +36,6 @@ import org.apache.solr.analytics.request.ExpressionRequest;
 import org.apache.solr.analytics.util.AnalyticsParams;
 import org.apache.solr.analytics.util.valuesource.AbsoluteValueDoubleFunction;
 import org.apache.solr.analytics.util.valuesource.AddDoubleFunction;
-import org.apache.solr.analytics.util.valuesource.ConcatStringFunction;
 import org.apache.solr.analytics.util.valuesource.ConstDateSource;
 import org.apache.solr.analytics.util.valuesource.ConstDoubleSource;
 import org.apache.solr.analytics.util.valuesource.ConstStringSource;
@@ -64,6 +63,7 @@ import org.apache.solr.schema.TrieDoubleField;
 import org.apache.solr.schema.TrieFloatField;
 import org.apache.solr.schema.TrieIntField;
 import org.apache.solr.schema.TrieLongField;
+import org.apache.solr.search.function.ConcatStringFunction;
 import org.apache.solr.util.DateMathParser;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParams.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParams.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParams.java
index b9da1cf..f6716ff 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParams.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParams.java
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.solr.common.params.FacetParams.FacetRangeInclude;
 import org.apache.solr.common.params.FacetParams.FacetRangeOther;
+import org.apache.solr.search.function.ConcatStringFunction;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
@@ -106,7 +107,7 @@ public interface AnalyticsParams {
   //Strings
   final static String CONSTANT_STRING = "const_str";
   final static String REVERSE = "rev";
-  final static String CONCATENATE = "concat";
+  final static String CONCATENATE = ConcatStringFunction.NAME;
   public static final Set<String> STRING_OPERATION_SET = Collections.unmodifiableSet(Sets.newLinkedHashSet(Lists.newArrayList(CONSTANT_STRING,REVERSE,CONCATENATE)));
   
   // Field Source Wrappers

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/ConcatStringFunction.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/ConcatStringFunction.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/ConcatStringFunction.java
deleted file mode 100644
index b1e1dc0..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/ConcatStringFunction.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.analytics.util.valuesource;
-
-import java.io.IOException;
-
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.solr.analytics.util.AnalyticsParams;
-
-/**
- * <code>ConcatStringFunction</code> concatenates the string values of its 
- * components in the order given.
- */
-public class ConcatStringFunction extends MultiStringFunction {
-  public final static String NAME = AnalyticsParams.CONCATENATE;
-
-  public ConcatStringFunction(ValueSource[] sources) {
-    super(sources);
-  }
-
-  protected String name() {
-    return NAME;
-  }
-
-  @Override
-  protected String func(int doc, FunctionValues[] valsArr) throws IOException {
-    StringBuilder sb = new StringBuilder();
-    for (FunctionValues val : valsArr) {
-      String v = val.strVal(doc);
-      if(v == null){
-        return null;
-      } else {
-        sb.append(v);
-      }
-    }
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/MultiStringFunction.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/MultiStringFunction.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/MultiStringFunction.java
deleted file mode 100644
index 4689d8b..0000000
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/MultiStringFunction.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * 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.analytics.util.valuesource;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.queries.function.FunctionValues;
-import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.queries.function.docvalues.StrDocValues;
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.mutable.MutableValue;
-import org.apache.lucene.util.mutable.MutableValueStr;
-
-/**
- * Abstract {@link ValueSource} implementation which wraps multiple ValueSources
- * and applies an extendible string function to their values.
- **/
-public abstract class MultiStringFunction extends ValueSource {
-  protected final ValueSource[] sources;
-  
-  public MultiStringFunction(ValueSource[] sources) {
-    this.sources = sources;
-  }
-
-  abstract protected String name();
-  abstract protected CharSequence func(int doc, FunctionValues[] valsArr) throws IOException;
-
-  @Override
-  public String description() {
-    StringBuilder sb = new StringBuilder();
-    sb.append(name()).append('(');
-    boolean firstTime=true;
-    for (ValueSource source : sources) {
-      if (firstTime) {
-        firstTime=false;
-      } else {
-        sb.append(',');
-      }
-      sb.append(source);
-    }
-    sb.append(')');
-    return sb.toString();
-  }
-
-  @Override
-  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
-    final FunctionValues[] valsArr = new FunctionValues[sources.length];
-    for (int i=0; i<sources.length; i++) {
-      valsArr[i] = sources[i].getValues(context, readerContext);
-    }
-
-    return new StrDocValues(this) {
-      @Override
-      public String strVal(int doc) throws IOException {
-        CharSequence cs = func(doc, valsArr);
-        return  cs != null ? cs.toString() : null;
-      }
-      
-      @Override
-      public boolean exists(int doc) throws IOException {
-        boolean exists = true;
-        for (FunctionValues val : valsArr) {
-          exists = exists & val.exists(doc);
-        }
-        return exists;
-      }
-      
-      @Override
-      public boolean bytesVal(int doc, BytesRefBuilder bytes) throws IOException {
-        bytes.clear();
-        CharSequence cs = func(doc, valsArr);
-        if( cs != null ){
-          bytes.copyChars(func(doc,valsArr));
-          return true;
-        } else {
-          return false;
-        }
-      }
-      
-      @Override
-      public String toString(int doc) throws IOException {
-        StringBuilder sb = new StringBuilder();
-        sb.append(name()).append('(');
-        boolean firstTime=true;
-        for (FunctionValues vals : valsArr) {
-          if (firstTime) {
-            firstTime=false;
-          } else {
-            sb.append(',');
-          }
-          sb.append(vals.toString(doc));
-        }
-        sb.append(')');
-        return sb.toString();
-      }
-
-      @Override
-      public ValueFiller getValueFiller() {
-        return new ValueFiller() {
-          private final MutableValueStr mval = new MutableValueStr();
-
-          @Override
-          public MutableValue getValue() {
-            return mval;
-          }
-
-          @Override
-          public void fillValue(int doc) throws IOException {
-            mval.exists = bytesVal(doc, mval.value);
-          }
-        };
-      }
-    };
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (getClass() != o.getClass()) return false;
-    MultiStringFunction other = (MultiStringFunction)o;
-    return this.name().equals(other.name())
-            && Arrays.equals(this.sources, other.sources);
-  }
-
-  @Override
-  public int hashCode() {
-    return Arrays.hashCode(sources) + name().hashCode();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
index 517b86f..b599272 100644
--- a/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
+++ b/solr/core/src/java/org/apache/solr/search/ValueSourceParser.java
@@ -64,6 +64,7 @@ import org.apache.solr.search.facet.SumsqAgg;
 import org.apache.solr.search.facet.UniqueAgg;
 import org.apache.solr.search.facet.VarianceAgg;
 import org.apache.solr.search.function.CollapseScoreFunction;
+import org.apache.solr.search.function.ConcatStringFunction;
 import org.apache.solr.search.function.OrdFieldSource;
 import org.apache.solr.search.function.ReverseOrdFieldSource;
 import org.apache.solr.search.function.SolrComparisonBoolFunction;
@@ -932,6 +933,14 @@ public abstract class ValueSourceParser implements NamedListInitializedPlugin {
       }
     });
 
+    addParser("concat", new ValueSourceParser() {
+      @Override
+      public ValueSource parse(FunctionQParser fp) throws SyntaxError {
+        List<ValueSource> sources = fp.parseValueSourceList();
+        return new ConcatStringFunction(sources.toArray(new ValueSource[sources.size()]));
+      }
+    });
+
 
     addParser("agg", new ValueSourceParser() {
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/core/src/java/org/apache/solr/search/function/ConcatStringFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/ConcatStringFunction.java b/solr/core/src/java/org/apache/solr/search/function/ConcatStringFunction.java
new file mode 100644
index 0000000..9162342
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/function/ConcatStringFunction.java
@@ -0,0 +1,53 @@
+/*
+ * 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.search.function;
+
+import java.io.IOException;
+
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+
+/**
+ * <code>ConcatStringFunction</code> concatenates the string values of its 
+ * components in the order given.
+ */
+public class ConcatStringFunction extends MultiStringFunction {
+  public final static String NAME = "concat";
+
+  public ConcatStringFunction(ValueSource[] sources) {
+    super(sources);
+  }
+
+  protected String name() {
+    return NAME;
+  }
+
+  @Override
+  protected String func(int doc, FunctionValues[] valsArr) throws IOException {
+    StringBuilder sb = new StringBuilder();
+    for (FunctionValues val : valsArr) {
+      String v = val.strVal(doc);
+      if(v == null){
+        return null;
+      } else {
+        sb.append(v);
+      }
+    }
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/core/src/java/org/apache/solr/search/function/MultiStringFunction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/function/MultiStringFunction.java b/solr/core/src/java/org/apache/solr/search/function/MultiStringFunction.java
new file mode 100644
index 0000000..c4aef6c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/function/MultiStringFunction.java
@@ -0,0 +1,146 @@
+/*
+ * 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.search.function;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.StrDocValues;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.mutable.MutableValue;
+import org.apache.lucene.util.mutable.MutableValueStr;
+
+/**
+ * Abstract {@link ValueSource} implementation which wraps multiple ValueSources
+ * and applies an extendible string function to their values.
+ **/
+public abstract class MultiStringFunction extends ValueSource {
+  protected final ValueSource[] sources;
+  
+  public MultiStringFunction(ValueSource[] sources) {
+    this.sources = sources;
+  }
+
+  abstract protected String name();
+  abstract protected CharSequence func(int doc, FunctionValues[] valsArr) throws IOException;
+
+  @Override
+  public String description() {
+    StringBuilder sb = new StringBuilder();
+    sb.append(name()).append('(');
+    boolean firstTime=true;
+    for (ValueSource source : sources) {
+      if (firstTime) {
+        firstTime=false;
+      } else {
+        sb.append(',');
+      }
+      sb.append(source);
+    }
+    sb.append(')');
+    return sb.toString();
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    final FunctionValues[] valsArr = new FunctionValues[sources.length];
+    for (int i=0; i<sources.length; i++) {
+      valsArr[i] = sources[i].getValues(context, readerContext);
+    }
+
+    return new StrDocValues(this) {
+      @Override
+      public String strVal(int doc) throws IOException {
+        CharSequence cs = func(doc, valsArr);
+        return  cs != null ? cs.toString() : null;
+      }
+      
+      @Override
+      public boolean exists(int doc) throws IOException {
+        boolean exists = true;
+        for (FunctionValues val : valsArr) {
+          exists = exists & val.exists(doc);
+        }
+        return exists;
+      }
+      
+      @Override
+      public boolean bytesVal(int doc, BytesRefBuilder bytes) throws IOException {
+        bytes.clear();
+        CharSequence cs = func(doc, valsArr);
+        if( cs != null ){
+          bytes.copyChars(func(doc,valsArr));
+          return true;
+        } else {
+          return false;
+        }
+      }
+      
+      @Override
+      public String toString(int doc) throws IOException {
+        StringBuilder sb = new StringBuilder();
+        sb.append(name()).append('(');
+        boolean firstTime=true;
+        for (FunctionValues vals : valsArr) {
+          if (firstTime) {
+            firstTime=false;
+          } else {
+            sb.append(',');
+          }
+          sb.append(vals.toString(doc));
+        }
+        sb.append(')');
+        return sb.toString();
+      }
+
+      @Override
+      public ValueFiller getValueFiller() {
+        return new ValueFiller() {
+          private final MutableValueStr mval = new MutableValueStr();
+
+          @Override
+          public MutableValue getValue() {
+            return mval;
+          }
+
+          @Override
+          public void fillValue(int doc) throws IOException {
+            mval.exists = bytesVal(doc, mval.value);
+          }
+        };
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (getClass() != o.getClass()) return false;
+    MultiStringFunction other = (MultiStringFunction)o;
+    return this.name().equals(other.name())
+            && Arrays.equals(this.sources, other.sources);
+  }
+
+  @Override
+  public int hashCode() {
+    return Arrays.hashCode(sources) + name().hashCode();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
index 0a9059c..aece15e 100644
--- a/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
+++ b/solr/core/src/test/org/apache/solr/search/QueryEqualityTest.java
@@ -862,6 +862,21 @@ public class QueryEqualityTest extends SolrTestCaseJ4 {
     }
   }
 
+  public void testFuncConcat() throws Exception {
+    SolrQueryRequest req = req("myField","bar_f","myOtherField","bar_t");
+
+    try {
+      assertFuncEquals(req,
+          "concat(bar_f,bar_t)",
+          "concat($myField,bar_t)",
+          "concat(bar_f,$myOtherField)",
+          "concat($myField,$myOtherField)");
+
+    } finally {
+      req.close();
+    }
+  }
+
   public void testFuncSingleValueMathFuncs() throws Exception {
     SolrQueryRequest req = req("myVal","45", "myField","foo_i");
     for (final String func : new String[] {"abs","rad","deg","sqrt","cbrt",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
index e77c4ee..675123c 100644
--- a/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
+++ b/solr/core/src/test/org/apache/solr/search/function/TestFunctionQuery.java
@@ -773,6 +773,25 @@ public class TestFunctionQuery extends SolrTestCaseJ4 {
 
   }
 
+  public void testConcatFunction() {
+    clearIndex();
+  
+    assertU(adoc("id", "1", "field1_t", "buzz", "field2_t", "word"));
+    assertU(adoc("id", "2", "field1_t", "1", "field2_t", "2","field4_t", "4"));
+    assertU(commit());
+  
+    assertQ(req("q","id:1",
+        "fl","field:concat(field1_t,field2_t)"),
+        " //str[@name='field']='buzzword'");
+  
+    assertQ(req("q","id:2",
+        "fl","field:concat(field1_t,field2_t,field4_t)"),
+        " //str[@name='field']='124'");
+  
+    assertQ(req("q","id:1",
+        "fl","field:def(concat(field3_t, field4_t), 'defValue')"),
+        " //str[@name='field']='defValue'");
+   }
 
   @Test
   public void testPseudoFieldFunctions() throws Exception {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/357f4dfb/solr/solr-ref-guide/src/function-queries.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/function-queries.adoc b/solr/solr-ref-guide/src/function-queries.adoc
index 0a628c1..624406f 100644
--- a/solr/solr-ref-guide/src/function-queries.adoc
+++ b/solr/solr-ref-guide/src/function-queries.adoc
@@ -86,6 +86,7 @@ The table below summarizes the functions available for function queries.
 |===
 |Function |Description |Syntax Examples
 |abs |Returns the absolute value of the specified value or function. |`abs(x)` `abs(-5)`
+|concat(v,f..)|concatenates the given string fields, literals and other functions |`concat(name," ",$param,def(opt,"-"))`
 |"constant" |Specifies a floating point constant. |`1.5`
 |def |`def` is short for default. Returns the value of field "field", or if the field does not exist, returns the default value specified. and yields the first value where `exists()==true`.) |`def(rating,5):` This `def()` function returns the rating, or if no rating specified in the doc, returns 5 `def(myfield, 1.0):` equivalent to `if(exists(myfield),myfield,1.0)`
 |div |Divides one value or function by another. div(x,y) divides x by y. |`div(1,y)` `div(sum(x,100),max(y,1))`