You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jb...@apache.org on 2018/04/02 17:11:36 UTC

[1/2] lucene-solr:branch_7x: SOLR-12174: Refactor Streaming Expression function registration

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x ffd46c8a7 -> 4d3149d43


SOLR-12174: Refactor Streaming Expression function registration


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

Branch: refs/heads/branch_7x
Commit: 8cb52a272cc8c80421783065670b680e5b0de3d6
Parents: ffd46c8
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Apr 2 12:31:44 2018 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Apr 2 12:57:09 2018 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/GraphHandler.java   |  58 +---
 .../org/apache/solr/handler/StreamHandler.java  | 278 +-----------------
 .../org/apache/solr/client/solrj/io/Lang.java   | 292 +++++++++++++++++++
 .../apache/solr/client/solrj/io/TestLang.java   |  94 ++++++
 4 files changed, 397 insertions(+), 325 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8cb52a27/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
index 62ad79c..cb4f8d5 100644
--- a/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/GraphHandler.java
@@ -25,23 +25,13 @@ import java.util.Map;
 import java.util.Map.Entry;
 
 import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.Lang;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
-import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
 import org.apache.solr.client.solrj.io.graph.Traversal;
-import org.apache.solr.client.solrj.io.ops.ConcatOperation;
-import org.apache.solr.client.solrj.io.ops.DistinctOperation;
-import org.apache.solr.client.solrj.io.ops.GroupOperation;
-import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.*;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.Expressible;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -57,7 +47,6 @@ import org.apache.solr.util.plugin.SolrCoreAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.solr.common.params.CommonParams.SORT;
 
 /**
  * @since 6.1.0
@@ -97,50 +86,7 @@ public class GraphHandler extends RequestHandlerBase implements SolrCoreAware, P
       streamFactory.withDefaultZkHost(defaultZkhost);
     }
 
-    streamFactory
-        // streams
-        .withFunctionName("search", CloudSolrStream.class)
-        .withFunctionName("merge", MergeStream.class)
-        .withFunctionName("unique", UniqueStream.class)
-        .withFunctionName("top", RankStream.class)
-        .withFunctionName("group", GroupOperation.class)
-        .withFunctionName("reduce", ReducerStream.class)
-        .withFunctionName("parallel", ParallelStream.class)
-        .withFunctionName("rollup", RollupStream.class)
-        .withFunctionName("stats", StatsStream.class)
-        .withFunctionName("innerJoin", InnerJoinStream.class)
-        .withFunctionName("leftOuterJoin", LeftOuterJoinStream.class)
-        .withFunctionName("hashJoin", HashJoinStream.class)
-        .withFunctionName("outerHashJoin", OuterHashJoinStream.class)
-        .withFunctionName("facet", FacetStream.class)
-        .withFunctionName("update", UpdateStream.class)
-        .withFunctionName("jdbc", JDBCStream.class)
-        .withFunctionName("intersect", IntersectStream.class)
-        .withFunctionName("select", SelectStream.class)
-        .withFunctionName("complement", ComplementStream.class)
-        .withFunctionName("daemon", DaemonStream.class)
-        .withFunctionName("topic", TopicStream.class)
-        .withFunctionName("shortestPath", ShortestPathStream.class)
-        .withFunctionName("gatherNodes", GatherNodesStream.class)
-        .withFunctionName("nodes", GatherNodesStream.class)
-        .withFunctionName(SORT, SortStream.class)
-        .withFunctionName("scoreNodes", ScoreNodesStream.class)
-        .withFunctionName("random", RandomStream.class)
-
-        // metrics
-        .withFunctionName("min", MinMetric.class)
-        .withFunctionName("max", MaxMetric.class)
-        .withFunctionName("avg", MeanMetric.class)
-        .withFunctionName("sum", SumMetric.class)
-        .withFunctionName("count", CountMetric.class)
-
-            // tuple manipulation operations
-        .withFunctionName("replace", ReplaceOperation.class)
-        .withFunctionName("concat", ConcatOperation.class)
-
-            // stream reduction operations
-        .withFunctionName("group", GroupOperation.class)
-        .withFunctionName("distinct", DistinctOperation.class);
+    Lang.register(streamFactory);
 
     // This pulls all the overrides and additions from the config
     Object functionMappingsObj = initArgs.get("streamFunctions");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8cb52a27/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
index 1b34d85..ffe66de 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -29,14 +29,8 @@ import java.util.Map;
 import org.apache.solr.client.solrj.io.ModelCache;
 import org.apache.solr.client.solrj.io.SolrClientCache;
 import org.apache.solr.client.solrj.io.Tuple;
+import org.apache.solr.client.solrj.io.Lang;
 import org.apache.solr.client.solrj.io.comp.StreamComparator;
-import org.apache.solr.client.solrj.io.eval.*;
-import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
-import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
-import org.apache.solr.client.solrj.io.ops.ConcatOperation;
-import org.apache.solr.client.solrj.io.ops.DistinctOperation;
-import org.apache.solr.client.solrj.io.ops.GroupOperation;
-import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
 import org.apache.solr.client.solrj.io.stream.*;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation;
 import org.apache.solr.client.solrj.io.stream.expr.Explanation.ExpressionType;
@@ -46,11 +40,6 @@ import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionNamedParameter;
 import org.apache.solr.client.solrj.io.stream.expr.StreamExpressionParser;
 import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
-import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
-import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -68,7 +57,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.solr.common.params.CommonParams.ID;
-import static org.apache.solr.common.params.CommonParams.SORT;
 
 /**
  * @since 5.1.0
@@ -118,262 +106,14 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
           clientCache);
     }
 
-    streamFactory
-        // source streams
-        .withFunctionName("search", CloudSolrStream.class)
-        .withFunctionName("facet", FacetStream.class)
-        .withFunctionName("update", UpdateStream.class)
-        .withFunctionName("jdbc", JDBCStream.class)
-        .withFunctionName("topic", TopicStream.class)
-        .withFunctionName("commit", CommitStream.class)
-        .withFunctionName("random", RandomStream.class)
-        .withFunctionName("knnSearch", KnnStream.class)
-
-        // decorator streams
-        .withFunctionName("merge", MergeStream.class)
-        .withFunctionName("unique", UniqueStream.class)
-        .withFunctionName("top", RankStream.class)
-        .withFunctionName("group", GroupOperation.class)
-        .withFunctionName("reduce", ReducerStream.class)
-        .withFunctionName("parallel", ParallelStream.class)
-        .withFunctionName("rollup", RollupStream.class)
-        .withFunctionName("stats", StatsStream.class)
-        .withFunctionName("innerJoin", InnerJoinStream.class)
-        .withFunctionName("leftOuterJoin", LeftOuterJoinStream.class)
-        .withFunctionName("hashJoin", HashJoinStream.class)
-        .withFunctionName("outerHashJoin", OuterHashJoinStream.class)
-        .withFunctionName("intersect", IntersectStream.class)
-        .withFunctionName("complement", ComplementStream.class)
-        .withFunctionName(SORT, SortStream.class)
-        .withFunctionName("train", TextLogitStream.class)
-        .withFunctionName("features", FeaturesSelectionStream.class)
-        .withFunctionName("daemon", DaemonStream.class)
-        .withFunctionName("shortestPath", ShortestPathStream.class)
-        .withFunctionName("gatherNodes", GatherNodesStream.class)
-        .withFunctionName("nodes", GatherNodesStream.class)
-        .withFunctionName("select", SelectStream.class)
-        .withFunctionName("shortestPath", ShortestPathStream.class)
-        .withFunctionName("gatherNodes", GatherNodesStream.class)
-        .withFunctionName("nodes", GatherNodesStream.class)
-        .withFunctionName("scoreNodes", ScoreNodesStream.class)
-        .withFunctionName("model", ModelStream.class)
-        .withFunctionName("classify", ClassifyStream.class)
-        .withFunctionName("fetch", FetchStream.class)
-        .withFunctionName("executor", ExecutorStream.class)
-        .withFunctionName("null", NullStream.class)
-        .withFunctionName("priority", PriorityStream.class)
-        .withFunctionName("significantTerms", SignificantTermsStream.class)
-        .withFunctionName("cartesianProduct", CartesianProductStream.class)
-        .withFunctionName("shuffle", ShuffleStream.class)
-        .withFunctionName("calc", CalculatorStream.class)
-        .withFunctionName("eval", EvalStream.class)
-        .withFunctionName("echo", EchoStream.class)
-        .withFunctionName("cell", CellStream.class)
-        .withFunctionName("list", ListStream.class)
-        .withFunctionName("let", LetStream.class)
-        .withFunctionName("get", GetStream.class)
-        .withFunctionName("timeseries", TimeSeriesStream.class)
-        .withFunctionName("tuple", TupStream.class)
-        .withFunctionName("sql", SqlStream.class)
-
-        // metrics
-        .withFunctionName("min", MinMetric.class)
-        .withFunctionName("max", MaxMetric.class)
-        .withFunctionName("avg", MeanMetric.class)
-        .withFunctionName("sum", SumMetric.class)
-        .withFunctionName("count", CountMetric.class)
-
-        // tuple manipulation operations
-        .withFunctionName("replace", ReplaceOperation.class)
-        .withFunctionName("concat", ConcatOperation.class)
-
-        // stream reduction operations
-        .withFunctionName("group", GroupOperation.class)
-        .withFunctionName("distinct", DistinctOperation.class)
-        .withFunctionName("having", HavingStream.class)
-
-        // Stream Evaluators
-        .withFunctionName("val", RawValueEvaluator.class)
-
-        // New Evaluators
-        .withFunctionName("anova", AnovaEvaluator.class)
-        .withFunctionName("array", ArrayEvaluator.class)
-        .withFunctionName("col", ColumnEvaluator.class)
-        .withFunctionName("conv", ConvolutionEvaluator.class)
-        .withFunctionName("copyOfRange", CopyOfRangeEvaluator.class)
-        .withFunctionName("copyOf", CopyOfEvaluator.class)
-        .withFunctionName("cov", CovarianceEvaluator.class)
-        .withFunctionName("corr", CorrelationEvaluator.class)
-        .withFunctionName("describe", DescribeEvaluator.class)
-        .withFunctionName("distance", DistanceEvaluator.class)
-        .withFunctionName("empiricalDistribution", EmpiricalDistributionEvaluator.class)
-        .withFunctionName("finddelay", FindDelayEvaluator.class)
-        .withFunctionName("hist", HistogramEvaluator.class)
-        .withFunctionName("length", LengthEvaluator.class)
-        .withFunctionName("movingAvg", MovingAverageEvaluator.class)
-        .withFunctionName("standardize", NormalizeEvaluator.class)
-        .withFunctionName("percentile", PercentileEvaluator.class)
-        .withFunctionName("predict", PredictEvaluator.class)
-        .withFunctionName("rank", RankEvaluator.class)
-        .withFunctionName("regress", RegressionEvaluator.class)
-        .withFunctionName("rev", ReverseEvaluator.class)
-        .withFunctionName("scale", ScaleEvaluator.class)
-        .withFunctionName("sequence", SequenceEvaluator.class)
-        .withFunctionName("addAll", AppendEvaluator.class)
-        .withFunctionName("append", AppendEvaluator.class)
-        .withFunctionName("plot", PlotStream.class)
-        .withFunctionName("normalDistribution", NormalDistributionEvaluator.class)
-        .withFunctionName("uniformDistribution", UniformDistributionEvaluator.class)
-        .withFunctionName("sample", SampleEvaluator.class)
-        .withFunctionName("kolmogorovSmirnov", KolmogorovSmirnovEvaluator.class)
-        .withFunctionName("ks", KolmogorovSmirnovEvaluator.class)
-        .withFunctionName("asc", AscEvaluator.class)
-        .withFunctionName("cumulativeProbability", CumulativeProbabilityEvaluator.class)
-        .withFunctionName("ebeAdd", EBEAddEvaluator.class)
-        .withFunctionName("ebeSubtract", EBESubtractEvaluator.class)
-        .withFunctionName("ebeMultiply", EBEMultiplyEvaluator.class)
-        .withFunctionName("ebeDivide", EBEDivideEvaluator.class)
-        .withFunctionName("dotProduct", DotProductEvaluator.class)
-        .withFunctionName("cosineSimilarity", CosineSimilarityEvaluator.class)
-        .withFunctionName("freqTable", FrequencyTableEvaluator.class)
-        .withFunctionName("uniformIntegerDistribution", UniformIntegerDistributionEvaluator.class)
-        .withFunctionName("binomialDistribution", BinomialDistributionEvaluator.class)
-        .withFunctionName("poissonDistribution", PoissonDistributionEvaluator.class)
-        .withFunctionName("enumeratedDistribution", EnumeratedDistributionEvaluator.class)
-        .withFunctionName("probability", ProbabilityEvaluator.class)
-        .withFunctionName("sumDifference", SumDifferenceEvaluator.class)
-        .withFunctionName("meanDifference", MeanDifferenceEvaluator.class)
-        .withFunctionName("primes", PrimesEvaluator.class)
-        .withFunctionName("factorial", FactorialEvaluator.class)
-        .withFunctionName("movingMedian", MovingMedianEvaluator.class)
-        .withFunctionName("binomialCoefficient", BinomialCoefficientEvaluator.class)
-        .withFunctionName("expMovingAvg", ExponentialMovingAverageEvaluator.class)
-        .withFunctionName("monteCarlo", MonteCarloEvaluator.class)
-        .withFunctionName("constantDistribution", ConstantDistributionEvaluator.class)
-        .withFunctionName("weibullDistribution", WeibullDistributionEvaluator.class)
-        .withFunctionName("mean", MeanEvaluator.class)
-        .withFunctionName("mode", ModeEvaluator.class)
-        .withFunctionName("logNormalDistribution", LogNormalDistributionEvaluator.class)
-        .withFunctionName("zipFDistribution", ZipFDistributionEvaluator.class)
-        .withFunctionName("gammaDistribution", GammaDistributionEvaluator.class)
-        .withFunctionName("betaDistribution", BetaDistributionEvaluator.class)
-        .withFunctionName("polyfit", PolyFitEvaluator.class)
-        .withFunctionName("harmonicFit", HarmonicFitEvaluator.class)
-        .withFunctionName("loess", LoessEvaluator.class)
-        .withFunctionName("matrix", MatrixEvaluator.class)
-        .withFunctionName("transpose", TransposeEvaluator.class)
-        .withFunctionName("unitize", UnitEvaluator.class)
-        .withFunctionName("triangularDistribution", TriangularDistributionEvaluator.class)
-        .withFunctionName("precision", PrecisionEvaluator.class)
-        .withFunctionName("minMaxScale", MinMaxScaleEvaluator.class)
-        .withFunctionName("markovChain", MarkovChainEvaluator.class)
-        .withFunctionName("grandSum", GrandSumEvaluator.class)
-        .withFunctionName("scalarAdd", ScalarAddEvaluator.class)
-        .withFunctionName("scalarSubtract", ScalarSubtractEvaluator.class)
-        .withFunctionName("scalarMultiply", ScalarMultiplyEvaluator.class)
-        .withFunctionName("scalarDivide", ScalarDivideEvaluator.class)
-        .withFunctionName("sumRows", SumRowsEvaluator.class)
-        .withFunctionName("sumColumns", SumColumnsEvaluator.class)
-        .withFunctionName("diff", TimeDifferencingEvaluator.class)
-        .withFunctionName("corrPValues", CorrelationSignificanceEvaluator.class)
-        .withFunctionName("normalizeSum", NormalizeSumEvaluator.class)
-        .withFunctionName("geometricDistribution", GeometricDistributionEvaluator.class)
-        .withFunctionName("olsRegress", OLSRegressionEvaluator.class)
-        .withFunctionName("derivative", DerivativeEvaluator.class)
-        .withFunctionName("spline", SplineEvaluator.class)
-        .withFunctionName("ttest", TTestEvaluator.class)
-        .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
-        .withFunctionName("multiVariateNormalDistribution", MultiVariateNormalDistributionEvaluator.class)
-        .withFunctionName("integrate", IntegrateEvaluator.class)
-        .withFunctionName("density", DensityEvaluator.class)
-        .withFunctionName("mannWhitney", MannWhitneyUEvaluator.class)
-        .withFunctionName("sumSq", SumSqEvaluator.class)
-        .withFunctionName("akima", AkimaEvaluator.class)
-        .withFunctionName("lerp", LerpEvaluator.class)
-        .withFunctionName("chiSquareDataSet", ChiSquareDataSetEvaluator.class)
-        .withFunctionName("gtestDataSet", GTestDataSetEvaluator.class)
-        .withFunctionName("termVectors", TermVectorsEvaluator.class)
-        .withFunctionName("getColumnLabels", GetColumnLabelsEvaluator.class)
-        .withFunctionName("getRowLabels", GetRowLabelsEvaluator.class)
-        .withFunctionName("getAttribute", GetAttributeEvaluator.class)
-        .withFunctionName("kmeans", KmeansEvaluator.class)
-        .withFunctionName("getCentroids", GetCentroidsEvaluator.class)
-        .withFunctionName("getCluster", GetClusterEvaluator.class)
-        .withFunctionName("topFeatures", TopFeaturesEvaluator.class)
-        .withFunctionName("featureSelect", FeatureSelectEvaluator.class)
-        .withFunctionName("rowAt", RowAtEvaluator.class)
-        .withFunctionName("colAt", ColumnAtEvaluator.class)
-        .withFunctionName("setColumnLabels", SetColumnLabelsEvaluator.class)
-        .withFunctionName("setRowLabels", SetRowLabelsEvaluator.class)
-        .withFunctionName("knn", KnnEvaluator.class)
-        .withFunctionName("getAttributes", GetAttributesEvaluator.class)
-        .withFunctionName("indexOf", IndexOfEvaluator.class)
-        .withFunctionName("columnCount", ColumnCountEvaluator.class)
-        .withFunctionName("rowCount", RowCountEvaluator.class)
-        .withFunctionName("fuzzyKmeans", FuzzyKmeansEvaluator.class)
-        .withFunctionName("getMembershipMatrix", GetMembershipMatrixEvaluator.class)
-        .withFunctionName("multiKmeans", MultiKmeansEvaluator.class)
-        .withFunctionName("l2norm", NormEvaluator.class)
-        .withFunctionName("l1norm", L1NormEvaluator.class)
-        .withFunctionName("linfnorm", LInfNormEvaluator.class)
-        .withFunctionName("matrixMult", MatrixMultiplyEvaluator.class)
-        .withFunctionName("bicubicSpline", BicubicSplineEvaluator.class)
-
-        // Boolean Stream Evaluators
-
-        .withFunctionName("and", AndEvaluator.class)
-        .withFunctionName("eor", ExclusiveOrEvaluator.class)
-        .withFunctionName("eq", EqualToEvaluator.class)
-        .withFunctionName("gt", GreaterThanEvaluator.class)
-        .withFunctionName("gteq", GreaterThanEqualToEvaluator.class)
-        .withFunctionName("lt", LessThanEvaluator.class)
-        .withFunctionName("lteq", LessThanEqualToEvaluator.class)
-        .withFunctionName("not", NotEvaluator.class)
-        .withFunctionName("or", OrEvaluator.class)
-
-        // Date Time Evaluators
-        .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
-        .withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class)
-        .withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME, TemporalEvaluatorDay.class)
-        .withFunctionName(TemporalEvaluatorDayOfYear.FUNCTION_NAME, TemporalEvaluatorDayOfYear.class)
-        .withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorHour.class)
-        .withFunctionName(TemporalEvaluatorMinute.FUNCTION_NAME, TemporalEvaluatorMinute.class)
-        .withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME, TemporalEvaluatorSecond.class)
-        .withFunctionName(TemporalEvaluatorEpoch.FUNCTION_NAME, TemporalEvaluatorEpoch.class)
-        .withFunctionName(TemporalEvaluatorWeek.FUNCTION_NAME, TemporalEvaluatorWeek.class)
-        .withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME, TemporalEvaluatorQuarter.class)
-        .withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, TemporalEvaluatorDayOfQuarter.class)
-
-        // Number Stream Evaluators
-        .withFunctionName("abs", AbsoluteValueEvaluator.class)
-        .withFunctionName("add", AddEvaluator.class)
-        .withFunctionName("div", DivideEvaluator.class)
-        .withFunctionName("mult", MultiplyEvaluator.class)
-        .withFunctionName("sub", SubtractEvaluator.class)
-        .withFunctionName("log", NaturalLogEvaluator.class)
-        .withFunctionName("pow", PowerEvaluator.class)
-        .withFunctionName("mod", ModuloEvaluator.class)
-        .withFunctionName("ceil", CeilingEvaluator.class)
-        .withFunctionName("floor", FloorEvaluator.class)
-        .withFunctionName("sin", SineEvaluator.class)
-        .withFunctionName("asin", ArcSineEvaluator.class)
-        .withFunctionName("sinh", HyperbolicSineEvaluator.class)
-        .withFunctionName("cos", CosineEvaluator.class)
-        .withFunctionName("acos", ArcCosineEvaluator.class)
-        .withFunctionName("cosh", HyperbolicCosineEvaluator.class)
-        .withFunctionName("tan", TangentEvaluator.class)
-        .withFunctionName("atan", ArcTangentEvaluator.class)
-        .withFunctionName("tanh", HyperbolicTangentEvaluator.class)
-        .withFunctionName("round", RoundEvaluator.class)
-        .withFunctionName("sqrt", SquareRootEvaluator.class)
-        .withFunctionName("cbrt", CubedRootEvaluator.class)
-        .withFunctionName("coalesce", CoalesceEvaluator.class)
-        .withFunctionName("uuid", UuidEvaluator.class)
-
-        // Conditional Stream Evaluators
-        .withFunctionName("if", IfThenElseEvaluator.class)
-        .withFunctionName("analyze", AnalyzeEvaluator.class)
-        .withFunctionName("convert", ConversionEvaluator.class);
+    Lang.register(streamFactory);
+
+    /*
+    * Add the core functions. These are functions that rely directly on either Lucene or Solr
+    * capabilities that are not part of Lang.
+    */
+    streamFactory.withFunctionName("analyze",  AnalyzeEvaluator.class);
+    streamFactory.withFunctionName("classify", ClassifyStream.class);
 
     // This pulls all the overrides and additions from the config
     List<PluginInfo> pluginInfos = core.getSolrConfig().getPluginInfos(Expressible.class.getName());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8cb52a27/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..08ba211
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/Lang.java
@@ -0,0 +1,292 @@
+/*
+ * 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;
+
+import org.apache.solr.client.solrj.io.eval.*;
+import org.apache.solr.client.solrj.io.graph.GatherNodesStream;
+import org.apache.solr.client.solrj.io.graph.ShortestPathStream;
+import org.apache.solr.client.solrj.io.ops.ConcatOperation;
+import org.apache.solr.client.solrj.io.ops.DistinctOperation;
+import org.apache.solr.client.solrj.io.ops.GroupOperation;
+import org.apache.solr.client.solrj.io.ops.ReplaceOperation;
+import org.apache.solr.client.solrj.io.stream.*;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+import org.apache.solr.client.solrj.io.stream.metrics.CountMetric;
+import org.apache.solr.client.solrj.io.stream.metrics.MaxMetric;
+import org.apache.solr.client.solrj.io.stream.metrics.MeanMetric;
+import org.apache.solr.client.solrj.io.stream.metrics.MinMetric;
+import org.apache.solr.client.solrj.io.stream.metrics.SumMetric;
+
+public class Lang {
+
+  public static void register(StreamFactory streamFactory) {
+    streamFactory
+        // source streams
+        .withFunctionName("search", CloudSolrStream.class)
+        .withFunctionName("facet", FacetStream.class)
+        .withFunctionName("update", UpdateStream.class)
+        .withFunctionName("jdbc", JDBCStream.class)
+        .withFunctionName("topic", TopicStream.class)
+        .withFunctionName("commit", CommitStream.class)
+        .withFunctionName("random", RandomStream.class)
+        .withFunctionName("knnSearch", KnnStream.class)
+
+            // decorator streams
+        .withFunctionName("merge", MergeStream.class)
+        .withFunctionName("unique", UniqueStream.class)
+        .withFunctionName("top", RankStream.class)
+        .withFunctionName("group", GroupOperation.class)
+        .withFunctionName("reduce", ReducerStream.class)
+        .withFunctionName("parallel", ParallelStream.class)
+        .withFunctionName("rollup", RollupStream.class)
+        .withFunctionName("stats", StatsStream.class)
+        .withFunctionName("innerJoin", InnerJoinStream.class)
+        .withFunctionName("leftOuterJoin", LeftOuterJoinStream.class)
+        .withFunctionName("hashJoin", HashJoinStream.class)
+        .withFunctionName("outerHashJoin", OuterHashJoinStream.class)
+        .withFunctionName("intersect", IntersectStream.class)
+        .withFunctionName("complement", ComplementStream.class)
+        .withFunctionName("sort", SortStream.class)
+        .withFunctionName("train", TextLogitStream.class)
+        .withFunctionName("features", FeaturesSelectionStream.class)
+        .withFunctionName("daemon", DaemonStream.class)
+        .withFunctionName("shortestPath", ShortestPathStream.class)
+        .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
+        .withFunctionName("select", SelectStream.class)
+        .withFunctionName("shortestPath", ShortestPathStream.class)
+        .withFunctionName("gatherNodes", GatherNodesStream.class)
+        .withFunctionName("nodes", GatherNodesStream.class)
+        .withFunctionName("scoreNodes", ScoreNodesStream.class)
+        .withFunctionName("model", ModelStream.class)
+        .withFunctionName("fetch", FetchStream.class)
+        .withFunctionName("executor", ExecutorStream.class)
+        .withFunctionName("null", NullStream.class)
+        .withFunctionName("priority", PriorityStream.class)
+        .withFunctionName("significantTerms", SignificantTermsStream.class)
+        .withFunctionName("cartesianProduct", CartesianProductStream.class)
+        .withFunctionName("shuffle", ShuffleStream.class)
+        .withFunctionName("calc", CalculatorStream.class)
+        .withFunctionName("eval", EvalStream.class)
+        .withFunctionName("echo", EchoStream.class)
+        .withFunctionName("cell", CellStream.class)
+        .withFunctionName("list", ListStream.class)
+        .withFunctionName("let", LetStream.class)
+        .withFunctionName("get", GetStream.class)
+        .withFunctionName("timeseries", TimeSeriesStream.class)
+        .withFunctionName("tuple", TupStream.class)
+        .withFunctionName("sql", SqlStream.class)
+
+            // metrics
+        .withFunctionName("min", MinMetric.class)
+        .withFunctionName("max", MaxMetric.class)
+        .withFunctionName("avg", MeanMetric.class)
+        .withFunctionName("sum", SumMetric.class)
+        .withFunctionName("count", CountMetric.class)
+
+            // tuple manipulation operations
+        .withFunctionName("replace", ReplaceOperation.class)
+        .withFunctionName("concat", ConcatOperation.class)
+
+            // stream reduction operations
+        .withFunctionName("group", GroupOperation.class)
+        .withFunctionName("distinct", DistinctOperation.class)
+        .withFunctionName("having", HavingStream.class)
+
+            // Stream Evaluators
+        .withFunctionName("val", RawValueEvaluator.class)
+
+            // New Evaluators
+        .withFunctionName("anova", AnovaEvaluator.class)
+        .withFunctionName("array", ArrayEvaluator.class)
+        .withFunctionName("col", ColumnEvaluator.class)
+        .withFunctionName("conv", ConvolutionEvaluator.class)
+        .withFunctionName("copyOfRange", CopyOfRangeEvaluator.class)
+        .withFunctionName("copyOf", CopyOfEvaluator.class)
+        .withFunctionName("cov", CovarianceEvaluator.class)
+        .withFunctionName("corr", CorrelationEvaluator.class)
+        .withFunctionName("describe", DescribeEvaluator.class)
+        .withFunctionName("distance", DistanceEvaluator.class)
+        .withFunctionName("empiricalDistribution", EmpiricalDistributionEvaluator.class)
+        .withFunctionName("finddelay", FindDelayEvaluator.class)
+        .withFunctionName("hist", HistogramEvaluator.class)
+        .withFunctionName("length", LengthEvaluator.class)
+        .withFunctionName("movingAvg", MovingAverageEvaluator.class)
+        .withFunctionName("standardize", NormalizeEvaluator.class)
+        .withFunctionName("percentile", PercentileEvaluator.class)
+        .withFunctionName("predict", PredictEvaluator.class)
+        .withFunctionName("rank", RankEvaluator.class)
+        .withFunctionName("regress", RegressionEvaluator.class)
+        .withFunctionName("rev", ReverseEvaluator.class)
+        .withFunctionName("scale", ScaleEvaluator.class)
+        .withFunctionName("sequence", SequenceEvaluator.class)
+        .withFunctionName("addAll", AppendEvaluator.class)
+        .withFunctionName("append", AppendEvaluator.class)
+        .withFunctionName("plot", PlotStream.class)
+        .withFunctionName("normalDistribution", NormalDistributionEvaluator.class)
+        .withFunctionName("uniformDistribution", UniformDistributionEvaluator.class)
+        .withFunctionName("sample", SampleEvaluator.class)
+        .withFunctionName("kolmogorovSmirnov", KolmogorovSmirnovEvaluator.class)
+        .withFunctionName("ks", KolmogorovSmirnovEvaluator.class)
+        .withFunctionName("asc", AscEvaluator.class)
+        .withFunctionName("cumulativeProbability", CumulativeProbabilityEvaluator.class)
+        .withFunctionName("ebeAdd", EBEAddEvaluator.class)
+        .withFunctionName("ebeSubtract", EBESubtractEvaluator.class)
+        .withFunctionName("ebeMultiply", EBEMultiplyEvaluator.class)
+        .withFunctionName("ebeDivide", EBEDivideEvaluator.class)
+        .withFunctionName("dotProduct", DotProductEvaluator.class)
+        .withFunctionName("cosineSimilarity", CosineSimilarityEvaluator.class)
+        .withFunctionName("freqTable", FrequencyTableEvaluator.class)
+        .withFunctionName("uniformIntegerDistribution", UniformIntegerDistributionEvaluator.class)
+        .withFunctionName("binomialDistribution", BinomialDistributionEvaluator.class)
+        .withFunctionName("poissonDistribution", PoissonDistributionEvaluator.class)
+        .withFunctionName("enumeratedDistribution", EnumeratedDistributionEvaluator.class)
+        .withFunctionName("probability", ProbabilityEvaluator.class)
+        .withFunctionName("sumDifference", SumDifferenceEvaluator.class)
+        .withFunctionName("meanDifference", MeanDifferenceEvaluator.class)
+        .withFunctionName("primes", PrimesEvaluator.class)
+        .withFunctionName("factorial", FactorialEvaluator.class)
+        .withFunctionName("movingMedian", MovingMedianEvaluator.class)
+        .withFunctionName("binomialCoefficient", BinomialCoefficientEvaluator.class)
+        .withFunctionName("expMovingAvg", ExponentialMovingAverageEvaluator.class)
+        .withFunctionName("monteCarlo", MonteCarloEvaluator.class)
+        .withFunctionName("constantDistribution", ConstantDistributionEvaluator.class)
+        .withFunctionName("weibullDistribution", WeibullDistributionEvaluator.class)
+        .withFunctionName("mean", MeanEvaluator.class)
+        .withFunctionName("mode", ModeEvaluator.class)
+        .withFunctionName("logNormalDistribution", LogNormalDistributionEvaluator.class)
+        .withFunctionName("zipFDistribution", ZipFDistributionEvaluator.class)
+        .withFunctionName("gammaDistribution", GammaDistributionEvaluator.class)
+        .withFunctionName("betaDistribution", BetaDistributionEvaluator.class)
+        .withFunctionName("polyfit", PolyFitEvaluator.class)
+        .withFunctionName("harmonicFit", HarmonicFitEvaluator.class)
+        .withFunctionName("loess", LoessEvaluator.class)
+        .withFunctionName("matrix", MatrixEvaluator.class)
+        .withFunctionName("transpose", TransposeEvaluator.class)
+        .withFunctionName("unitize", UnitEvaluator.class)
+        .withFunctionName("triangularDistribution", TriangularDistributionEvaluator.class)
+        .withFunctionName("precision", PrecisionEvaluator.class)
+        .withFunctionName("minMaxScale", MinMaxScaleEvaluator.class)
+        .withFunctionName("markovChain", MarkovChainEvaluator.class)
+        .withFunctionName("grandSum", GrandSumEvaluator.class)
+        .withFunctionName("scalarAdd", ScalarAddEvaluator.class)
+        .withFunctionName("scalarSubtract", ScalarSubtractEvaluator.class)
+        .withFunctionName("scalarMultiply", ScalarMultiplyEvaluator.class)
+        .withFunctionName("scalarDivide", ScalarDivideEvaluator.class)
+        .withFunctionName("sumRows", SumRowsEvaluator.class)
+        .withFunctionName("sumColumns", SumColumnsEvaluator.class)
+        .withFunctionName("diff", TimeDifferencingEvaluator.class)
+        .withFunctionName("corrPValues", CorrelationSignificanceEvaluator.class)
+        .withFunctionName("normalizeSum", NormalizeSumEvaluator.class)
+        .withFunctionName("geometricDistribution", GeometricDistributionEvaluator.class)
+        .withFunctionName("olsRegress", OLSRegressionEvaluator.class)
+        .withFunctionName("derivative", DerivativeEvaluator.class)
+        .withFunctionName("spline", SplineEvaluator.class)
+        .withFunctionName("ttest", TTestEvaluator.class)
+        .withFunctionName("pairedTtest", PairedTTestEvaluator.class)
+        .withFunctionName("multiVariateNormalDistribution", MultiVariateNormalDistributionEvaluator.class)
+        .withFunctionName("integrate", IntegrateEvaluator.class)
+        .withFunctionName("density", DensityEvaluator.class)
+        .withFunctionName("mannWhitney", MannWhitneyUEvaluator.class)
+        .withFunctionName("sumSq", SumSqEvaluator.class)
+        .withFunctionName("akima", AkimaEvaluator.class)
+        .withFunctionName("lerp", LerpEvaluator.class)
+        .withFunctionName("chiSquareDataSet", ChiSquareDataSetEvaluator.class)
+        .withFunctionName("gtestDataSet", GTestDataSetEvaluator.class)
+        .withFunctionName("termVectors", TermVectorsEvaluator.class)
+        .withFunctionName("getColumnLabels", GetColumnLabelsEvaluator.class)
+        .withFunctionName("getRowLabels", GetRowLabelsEvaluator.class)
+        .withFunctionName("getAttribute", GetAttributeEvaluator.class)
+        .withFunctionName("kmeans", KmeansEvaluator.class)
+        .withFunctionName("getCentroids", GetCentroidsEvaluator.class)
+        .withFunctionName("getCluster", GetClusterEvaluator.class)
+        .withFunctionName("topFeatures", TopFeaturesEvaluator.class)
+        .withFunctionName("featureSelect", FeatureSelectEvaluator.class)
+        .withFunctionName("rowAt", RowAtEvaluator.class)
+        .withFunctionName("colAt", ColumnAtEvaluator.class)
+        .withFunctionName("setColumnLabels", SetColumnLabelsEvaluator.class)
+        .withFunctionName("setRowLabels", SetRowLabelsEvaluator.class)
+        .withFunctionName("knn", KnnEvaluator.class)
+        .withFunctionName("getAttributes", GetAttributesEvaluator.class)
+        .withFunctionName("indexOf", IndexOfEvaluator.class)
+        .withFunctionName("columnCount", ColumnCountEvaluator.class)
+        .withFunctionName("rowCount", RowCountEvaluator.class)
+        .withFunctionName("fuzzyKmeans", FuzzyKmeansEvaluator.class)
+        .withFunctionName("getMembershipMatrix", GetMembershipMatrixEvaluator.class)
+        .withFunctionName("multiKmeans", MultiKmeansEvaluator.class)
+        .withFunctionName("l2norm", NormEvaluator.class)
+        .withFunctionName("l1norm", L1NormEvaluator.class)
+        .withFunctionName("linfnorm", LInfNormEvaluator.class)
+        .withFunctionName("matrixMult", MatrixMultiplyEvaluator.class)
+        .withFunctionName("bicubicSpline", BicubicSplineEvaluator.class)
+
+            // Boolean Stream Evaluators
+
+        .withFunctionName("and", AndEvaluator.class)
+        .withFunctionName("eor", ExclusiveOrEvaluator.class)
+        .withFunctionName("eq", EqualToEvaluator.class)
+        .withFunctionName("gt", GreaterThanEvaluator.class)
+        .withFunctionName("gteq", GreaterThanEqualToEvaluator.class)
+        .withFunctionName("lt", LessThanEvaluator.class)
+        .withFunctionName("lteq", LessThanEqualToEvaluator.class)
+        .withFunctionName("not", NotEvaluator.class)
+        .withFunctionName("or", OrEvaluator.class)
+
+            // Date Time Evaluators
+        .withFunctionName(TemporalEvaluatorYear.FUNCTION_NAME, TemporalEvaluatorYear.class)
+        .withFunctionName(TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorMonth.class)
+        .withFunctionName(TemporalEvaluatorDay.FUNCTION_NAME, TemporalEvaluatorDay.class)
+        .withFunctionName(TemporalEvaluatorDayOfYear.FUNCTION_NAME, TemporalEvaluatorDayOfYear.class)
+        .withFunctionName(TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorHour.class)
+        .withFunctionName(TemporalEvaluatorMinute.FUNCTION_NAME, TemporalEvaluatorMinute.class)
+        .withFunctionName(TemporalEvaluatorSecond.FUNCTION_NAME, TemporalEvaluatorSecond.class)
+        .withFunctionName(TemporalEvaluatorEpoch.FUNCTION_NAME, TemporalEvaluatorEpoch.class)
+        .withFunctionName(TemporalEvaluatorWeek.FUNCTION_NAME, TemporalEvaluatorWeek.class)
+        .withFunctionName(TemporalEvaluatorQuarter.FUNCTION_NAME, TemporalEvaluatorQuarter.class)
+        .withFunctionName(TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, TemporalEvaluatorDayOfQuarter.class)
+
+            // Number Stream Evaluators
+        .withFunctionName("abs", AbsoluteValueEvaluator.class)
+        .withFunctionName("add", AddEvaluator.class)
+        .withFunctionName("div", DivideEvaluator.class)
+        .withFunctionName("mult", MultiplyEvaluator.class)
+        .withFunctionName("sub", SubtractEvaluator.class)
+        .withFunctionName("log", NaturalLogEvaluator.class)
+        .withFunctionName("pow", PowerEvaluator.class)
+        .withFunctionName("mod", ModuloEvaluator.class)
+        .withFunctionName("ceil", CeilingEvaluator.class)
+        .withFunctionName("floor", FloorEvaluator.class)
+        .withFunctionName("sin", SineEvaluator.class)
+        .withFunctionName("asin", ArcSineEvaluator.class)
+        .withFunctionName("sinh", HyperbolicSineEvaluator.class)
+        .withFunctionName("cos", CosineEvaluator.class)
+        .withFunctionName("acos", ArcCosineEvaluator.class)
+        .withFunctionName("cosh", HyperbolicCosineEvaluator.class)
+        .withFunctionName("tan", TangentEvaluator.class)
+        .withFunctionName("atan", ArcTangentEvaluator.class)
+        .withFunctionName("tanh", HyperbolicTangentEvaluator.class)
+        .withFunctionName("round", RoundEvaluator.class)
+        .withFunctionName("sqrt", SquareRootEvaluator.class)
+        .withFunctionName("cbrt", CubedRootEvaluator.class)
+        .withFunctionName("coalesce", CoalesceEvaluator.class)
+        .withFunctionName("uuid", UuidEvaluator.class)
+
+            // Conditional Stream Evaluators
+        .withFunctionName("if", IfThenElseEvaluator.class)
+        .withFunctionName("convert", ConversionEvaluator.class);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8cb52a27/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..498cb2e
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/TestLang.java
@@ -0,0 +1,94 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.io.eval.*;
+import org.apache.solr.client.solrj.io.stream.expr.Expressible;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+
+import org.junit.Test;
+
+@Slow
+@LuceneTestCase.SuppressCodecs({"Lucene3x", "Lucene40","Lucene41","Lucene42","Lucene45"})
+public class TestLang extends LuceneTestCase {
+
+  private static final String[] allFunctions = {
+      "search", "facet", "update", "jdbc", "topic", "commit", "random", "knnSearch", "merge",
+      "unique", "top", "group", "reduce", "parallel", "rollup", "stats", "innerJoin",
+      "leftOuterJoin", "hashJoin", "outerHashJoin", "intersect", "complement", "sort",
+      "train", "features", "daemon", "shortestPath", "gatherNodes", "nodes",
+      "select", "shortestPath", "gatherNodes", "nodes", "scoreNodes", "model", "fetch",
+      "executor", "null", "priority", "significantTerms", "cartesianProduct",
+      "shuffle", "calc", "eval", "echo", "cell", "list", "let", "get", "timeseries", "tuple", "sql", "min",
+      "max", "avg", "sum", "count", "replace", "concat", "group", "distinct", "having",
+      "val", "anova", "array", "col", "conv", "copyOfRange", "copyOf", "cov", "corr", "describe", "distance", "empiricalDistribution",
+      "finddelay", "hist", "length", "movingAvg", "standardize", "percentile", "predict", "rank", "regress", "rev", "scale",
+      "sequence", "addAll", "append", "plot", "normalDistribution", "uniformDistribution", "sample", "kolmogorovSmirnov",
+      "ks", "asc", "cumulativeProbability", "ebeAdd", "ebeSubtract", "ebeMultiply", "ebeDivide",
+      "dotProduct", "cosineSimilarity", "freqTable", "uniformIntegerDistribution", "binomialDistribution",
+      "poissonDistribution", "enumeratedDistribution", "probability", "sumDifference", "meanDifference",
+      "primes", "factorial", "movingMedian", "binomialCoefficient", "expMovingAvg", "monteCarlo", "constantDistribution",
+      "weibullDistribution", "mean", "mode", "logNormalDistribution", "zipFDistribution", "gammaDistribution",
+      "betaDistribution", "polyfit", "harmonicFit", "loess", "matrix", "transpose", "unitize",
+      "triangularDistribution", "precision", "minMaxScale", "markovChain", "grandSum",
+      "scalarAdd", "scalarSubtract", "scalarMultiply", "scalarDivide", "sumRows",
+      "sumColumns", "diff", "corrPValues", "normalizeSum", "geometricDistribution", "olsRegress",
+      "derivative", "spline", "ttest", "pairedTtest", "multiVariateNormalDistribution", "integrate",
+      "density", "mannWhitney", "sumSq", "akima", "lerp", "chiSquareDataSet", "gtestDataSet",
+      "termVectors", "getColumnLabels", "getRowLabels", "getAttribute", "kmeans", "getCentroids",
+      "getCluster", "topFeatures", "featureSelect", "rowAt", "colAt", "setColumnLabels",
+      "setRowLabels", "knn", "getAttributes", "indexOf", "columnCount", "rowCount", "fuzzyKmeans",
+      "getMembershipMatrix", "multiKmeans", "l2norm", "l1norm", "linfnorm", "matrixMult",
+      "bicubicSpline", "and", "eor", "eq", "gt", "gteq", "lt", "lteq", "not", "or", TemporalEvaluatorYear.FUNCTION_NAME,
+       TemporalEvaluatorMonth.FUNCTION_NAME, TemporalEvaluatorDay.FUNCTION_NAME, TemporalEvaluatorDayOfYear.FUNCTION_NAME,
+       TemporalEvaluatorHour.FUNCTION_NAME, TemporalEvaluatorMinute.FUNCTION_NAME, TemporalEvaluatorSecond.FUNCTION_NAME,
+       TemporalEvaluatorEpoch.FUNCTION_NAME, TemporalEvaluatorWeek.FUNCTION_NAME, TemporalEvaluatorQuarter.FUNCTION_NAME,
+       TemporalEvaluatorDayOfQuarter.FUNCTION_NAME, "abs", "add", "div", "mult", "sub", "log", "pow",
+      "mod", "ceil", "floor", "sin", "asin", "sinh", "cos", "acos", "cosh", "tan", "atan", "tanh", "round", "sqrt",
+      "cbrt", "coalesce", "uuid", "if", "convert"};
+
+  @Test
+  public void testLang() {
+    List<String> functions = new ArrayList();
+    for(String f : allFunctions) {
+      functions.add(f);
+    }
+    StreamFactory factory = new StreamFactory();
+    Lang.register(factory);
+    Map<String,Class<? extends Expressible>> registeredFunctions = factory.getFunctionNames();
+
+    //Check that each function that is expected is registered.
+    for(String func : functions) {
+      assertTrue("Testing function:"+func, registeredFunctions.containsKey(func));
+    }
+
+    //Check that ech function that is registered is expected.
+    Set<String> keys = registeredFunctions.keySet();
+    for(String key : keys) {
+      assertTrue("Testing key:"+key, functions.contains(key));
+    }
+  }
+}


[2/2] lucene-solr:branch_7x: SOLR-12133: Fix precommit

Posted by jb...@apache.org.
SOLR-12133: Fix precommit


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

Branch: refs/heads/branch_7x
Commit: 4d3149d4344efed263d852829273d90d9b0a1c12
Parents: 8cb52a2
Author: Joel Bernstein <jb...@apache.org>
Authored: Mon Apr 2 12:53:12 2018 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Mon Apr 2 13:05:57 2018 -0400

----------------------------------------------------------------------
 .../apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d3149d4/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
index 48cae62..e88324e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/NodeMarkersRegistrationTest.java
@@ -28,7 +28,6 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;