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/08/27 00:02:30 UTC

lucene-solr:branch_7x: SOLR-12687: Add functions to cache data structures and mathematical models

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x da37ffb51 -> 34a8c023b


SOLR-12687: Add functions to cache data structures and mathematical models


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

Branch: refs/heads/branch_7x
Commit: 34a8c023bd773a384c820dae0e888621c36d1662
Parents: da37ffb
Author: Joel Bernstein <jb...@apache.org>
Authored: Wed Aug 22 22:47:04 2018 -0400
Committer: Joel Bernstein <jb...@apache.org>
Committed: Sun Aug 26 19:57:56 2018 -0400

----------------------------------------------------------------------
 .../org/apache/solr/handler/StreamHandler.java  |  4 +
 .../org/apache/solr/client/solrj/io/Lang.java   |  5 +-
 .../client/solrj/io/eval/GetCacheEvaluator.java | 57 +++++++++++++
 .../solrj/io/eval/ListCacheEvaluator.java       | 73 +++++++++++++++++
 .../client/solrj/io/eval/PutCacheEvaluator.java | 61 ++++++++++++++
 .../solrj/io/eval/RemoveCacheEvaluator.java     | 57 +++++++++++++
 .../client/solrj/io/stream/StreamContext.java   | 10 +++
 .../apache/solr/client/solrj/io/TestLang.java   |  2 +-
 .../solrj/io/stream/MathExpressionTest.java     | 85 ++++++++++++++++++++
 9 files changed, 352 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/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 e6ebc51..a447093 100644
--- a/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/StreamHandler.java
@@ -25,6 +25,8 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.solr.client.solrj.io.ModelCache;
 import org.apache.solr.client.solrj.io.SolrClientCache;
@@ -64,6 +66,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
 
   static SolrClientCache clientCache = new SolrClientCache();
   static ModelCache modelCache = null;
+  static ConcurrentMap objectCache = new ConcurrentHashMap();
   private SolrDefaultStreamFactory streamFactory = new SolrDefaultStreamFactory();
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private String coreName;
@@ -164,6 +167,7 @@ public class StreamHandler extends RequestHandlerBase implements SolrCoreAware,
     context.numWorkers = numWorkers;
     context.setSolrClientCache(clientCache);
     context.setModelCache(modelCache);
+    context.setObjectCache(objectCache);
     context.put("core", this.coreName);
     context.put("solr-core", req.getCore());
     tupleStream.setStreamContext(context);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/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
index c9e1326..69673ee 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
@@ -250,7 +250,10 @@ public class Lang {
         .withFunctionName("gaussfit", GaussFitEvaluator.class)
         .withFunctionName("outliers", OutliersEvaluator.class)
         .withFunctionName("stream", GetStream.class)
-
+        .withFunctionName("putCache", PutCacheEvaluator.class)
+        .withFunctionName("getCache", GetCacheEvaluator.class)
+        .withFunctionName("removeCache", RemoveCacheEvaluator.class)
+        .withFunctionName("listCache", ListCacheEvaluator.class)
         // Boolean Stream Evaluators
 
         .withFunctionName("and", AndEvaluator.class)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GetCacheEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GetCacheEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GetCacheEvaluator.java
new file mode 100644
index 0000000..e340d80
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/GetCacheEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+
+import java.util.Locale;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class GetCacheEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public GetCacheEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(2 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 3 values but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+    ConcurrentMap objectCache = this.streamContext.getObjectCache();
+    if(values.length == 2) {
+      String space = (String)values[0];
+      String key = (String)values[1];
+      space = space.replace("\"", "");
+      key = key.replace("\"", "");
+      ConcurrentMap spaceCache = (ConcurrentMap)objectCache.get(space);
+
+      if(spaceCache != null) {
+        return spaceCache.get(key);
+      }
+
+      return null;
+    } else {
+      throw new IOException("The getCache function requires two parameters: workspace and key");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ListCacheEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ListCacheEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ListCacheEvaluator.java
new file mode 100644
index 0000000..99c25e7
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/ListCacheEvaluator.java
@@ -0,0 +1,73 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+
+import java.util.Enumeration;
+import java.util.Locale;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentHashMap;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class ListCacheEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public ListCacheEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(containedEvaluators.size() > 1){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting at most 1 values but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+    ConcurrentMap objectCache = this.streamContext.getObjectCache();
+    List list = new ArrayList();
+
+    if(values.length == 0) {
+      ConcurrentHashMap m = (ConcurrentHashMap)objectCache;
+      Enumeration en = m.keys();
+      while(en.hasMoreElements()) {
+        list.add(en.nextElement());
+      }
+      return list;
+    } else if(values.length == 1) {
+      String space = (String)values[0];
+      space = space.replace("\"", "");
+      ConcurrentMap spaceCache = (ConcurrentMap)objectCache.get(space);
+      if(spaceCache != null) {
+        ConcurrentHashMap spaceMap = (ConcurrentHashMap)objectCache.get(space);
+        Enumeration en = spaceMap.keys();
+        while(en.hasMoreElements()) {
+          list.add(en.nextElement());
+        }
+        return list;
+      } else {
+        return list;
+      }
+    } else {
+      throw new IOException("The listCache function requires two parameters: workspace and key");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PutCacheEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PutCacheEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PutCacheEvaluator.java
new file mode 100644
index 0000000..dfd46f9
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/PutCacheEvaluator.java
@@ -0,0 +1,61 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+
+import java.util.Locale;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class PutCacheEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public PutCacheEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(3 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 3 values but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+    ConcurrentMap objectCache = this.streamContext.getObjectCache();
+    if(values.length == 3) {
+      String space = (String)values[0];
+      String key = (String)values[1];
+      space = space.replace("\"", "");
+      key = key.replace("\"", "");
+      Object value = values[2];
+      ConcurrentMap spaceCache = (ConcurrentMap)objectCache.get(space);
+      if(spaceCache == null) {
+        spaceCache = new ConcurrentHashMap();
+        objectCache.put(space, spaceCache);
+      }
+
+      spaceCache.put(key, value);
+      return value;
+    } else {
+      throw new IOException("The putCache function requires three parameters: workspace, key and value");
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RemoveCacheEvaluator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RemoveCacheEvaluator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RemoveCacheEvaluator.java
new file mode 100644
index 0000000..92fa3cd
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/eval/RemoveCacheEvaluator.java
@@ -0,0 +1,57 @@
+/*
+ * 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.eval;
+
+import java.io.IOException;
+
+import java.util.Locale;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.solr.client.solrj.io.stream.expr.StreamExpression;
+import org.apache.solr.client.solrj.io.stream.expr.StreamFactory;
+
+public class RemoveCacheEvaluator extends RecursiveObjectEvaluator implements ManyValueWorker {
+  protected static final long serialVersionUID = 1L;
+
+  public RemoveCacheEvaluator(StreamExpression expression, StreamFactory factory) throws IOException{
+    super(expression, factory);
+
+    if(2 != containedEvaluators.size()){
+      throw new IOException(String.format(Locale.ROOT,"Invalid expression %s - expecting exactly 3 values but found %d",expression,containedEvaluators.size()));
+    }
+  }
+
+  @Override
+  public Object doWork(Object... values) throws IOException {
+    ConcurrentMap objectCache = this.streamContext.getObjectCache();
+    if(values.length == 2) {
+      String space = (String)values[0];
+      String key = (String)values[1];
+      space = space.replace("\"", "");
+      key = key.replace("\"", "");
+      ConcurrentMap spaceCache = (ConcurrentMap)objectCache.get(space);
+
+      if(spaceCache != null) {
+        return spaceCache.remove(key);
+      }
+
+      return false;
+    } else {
+      throw new IOException("The removeCache function requires two parameters: workspace and key");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
index a548683..778aace 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/StreamContext.java
@@ -19,6 +19,7 @@ package org.apache.solr.client.solrj.io.stream;
 import java.io.Serializable;
 import java.util.Map;
 import java.util.HashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.solr.client.solrj.io.ModelCache;
 import org.apache.solr.client.solrj.io.SolrClientCache;
@@ -38,12 +39,21 @@ public class StreamContext implements Serializable{
   private Map entries = new HashMap();
   private Map tupleContext = new HashMap();
   private Map<String, Object> lets = new HashMap();
+  private ConcurrentMap objectCache;
   public int workerID;
   public int numWorkers;
   private SolrClientCache clientCache;
   private ModelCache modelCache;
   private StreamFactory streamFactory;
 
+  public ConcurrentMap getObjectCache() {
+    return this.objectCache;
+  }
+
+  public void setObjectCache(ConcurrentMap objectCache) {
+    this.objectCache = objectCache;
+  }
+
   public Map<String, Object> getLets(){
     return lets;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/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
index 3abd196..242f551 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
@@ -70,7 +70,7 @@ public class TestLang extends LuceneTestCase {
       "mod", "ceil", "floor", "sin", "asin", "sinh", "cos", "acos", "cosh", "tan", "atan", "tanh", "round", "sqrt",
       "cbrt", "coalesce", "uuid", "if", "convert", "valueAt", "memset", "fft", "ifft", "euclidean","manhattan",
       "earthMovers", "canberra", "chebyshev", "ones", "zeros", "setValue", "getValue", "knnRegress", "gaussfit",
-      "outliers", "stream"};
+      "outliers", "stream", "getCache", "putCache", "listCache", "removeCache"};
 
   @Test
   public void testLang() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34a8c023/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index 6565b76..468d0b3 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -2731,6 +2731,91 @@ public class MathExpressionTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testCache() throws Exception {
+    String cexpr = "putCache(space1, key1, dotProduct(array(2,4,6,8,10,12),array(1,2,3,4,5,6)))";
+    ModifiableSolrParams paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    String url = cluster.getJettySolrRunners().get(0).getBaseUrl().toString()+"/"+COLLECTIONORALIAS;
+    TupleStream solrStream = new SolrStream(url, paramsLoc);
+    StreamContext context = new StreamContext();
+    solrStream.setStreamContext(context);
+    List<Tuple> tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    Number dotProduct = (Number)tuples.get(0).get("return-value");
+    assertTrue(dotProduct.doubleValue() == 182);
+
+
+    cexpr = "getCache(space1, key1)";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    context = new StreamContext();
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    dotProduct = (Number)tuples.get(0).get("return-value");
+    assertTrue(dotProduct.doubleValue() == 182);
+
+    cexpr = "listCache(space1)";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    context = new StreamContext();
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    List<String> keys = (List<String>)tuples.get(0).get("return-value");
+    assertEquals(keys.size(), 1);
+    assertEquals(keys.get(0), "key1");
+
+    cexpr = "listCache()";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    context = new StreamContext();
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    keys = (List<String>)tuples.get(0).get("return-value");
+    assertEquals(keys.size(), 1);
+    assertEquals(keys.get(0), "space1");
+
+    cexpr = "removeCache(space1, key1)";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    context = new StreamContext();
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    dotProduct = (Number)tuples.get(0).get("return-value");
+    assertTrue(dotProduct.doubleValue() == 182);
+
+
+    cexpr = "listCache(space1)";
+    paramsLoc = new ModifiableSolrParams();
+    paramsLoc.set("expr", cexpr);
+    paramsLoc.set("qt", "/stream");
+    solrStream = new SolrStream(url, paramsLoc);
+    context = new StreamContext();
+    solrStream.setStreamContext(context);
+    tuples = getTuples(solrStream);
+    assertTrue(tuples.size() == 1);
+    keys = (List<String>)tuples.get(0).get("return-value");
+    assertEquals(keys.size(), 0);
+
+
+
+
+
+  }
+
+  @Test
   public void testExponentialMovingAverage() throws Exception {
     String cexpr = "expMovingAvg(array(22.27, 22.19, 22.08, 22.17, 22.18, 22.13, 22.23, 22.43, 22.24, 22.29, " +
                    "22.15, 22.39, 22.38, 22.61, 23.36, 24.05, 23.75, 23.83, 23.95, 23.63, 23.82, 23.87, 23.65, 23.19,"+