From commits-return-103074-archive-asf-public=cust-asf.ponee.io@lucene.apache.org Mon Aug 27 02:02:32 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id ADE61180672 for ; Mon, 27 Aug 2018 02:02:31 +0200 (CEST) Received: (qmail 52477 invoked by uid 500); 27 Aug 2018 00:02:30 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 52468 invoked by uid 99); 27 Aug 2018 00:02:30 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 27 Aug 2018 00:02:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 833B7DFC44; Mon, 27 Aug 2018 00:02:30 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jbernste@apache.org To: commits@lucene.apache.org Message-Id: <876e93c43ab74308a1a9c6332a48e825@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: lucene-solr:branch_7x: SOLR-12687: Add functions to cache data structures and mathematical models Date: Mon, 27 Aug 2018 00:02:30 +0000 (UTC) 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 Authored: Wed Aug 22 22:47:04 2018 -0400 Committer: Joel Bernstein 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 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 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 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 keys = (List)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)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)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,"+