diff --git a/solr/contrib/dataimporthandler/CHANGES.txt b/solr/contrib/dataimporthandler/CHANGES.txt index 4e6cb12486f..eb6d3afcc87 100644 --- a/solr/contrib/dataimporthandler/CHANGES.txt +++ b/solr/contrib/dataimporthandler/CHANGES.txt @@ -12,6 +12,13 @@ $Id$ (No Changes) +================== 3.6.0 ================== + +New Features +---------------------- +* SOLR-1499: Added SolrEntityProcessor that imports data from another Solr core or instance based on a specified query. + (Lance Norskog, Erik Hatcher, Pulkit Singhal, Ahmet Arslan, Luca Cavanna, Martijn van Groningen) + ================== 3.5.0 ================== Bug Fixes diff --git a/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java new file mode 100644 index 00000000000..c2c169e4e73 --- /dev/null +++ b/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrEntityProcessor.java @@ -0,0 +1,254 @@ +package org.apache.solr.handler.dataimport; + +/* + * 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. + */ + +import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE; +import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +import org.apache.commons.httpclient.HttpClient; +import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.SolrServer; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer; +import org.apache.solr.client.solrj.impl.XMLResponseParser; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrDocumentList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + *

+ * An implementation of {@link EntityProcessor} which fetches values from a + * separate Solr implementation using the SolrJ client library. Yield a row per + * Solr document. + *

+ *

+ * Limitations: + * All configuration is evaluated at the beginning; + * Only one query is walked; + *

+ */ +public class SolrEntityProcessor extends EntityProcessorBase { + + private static final Logger LOG = LoggerFactory.getLogger(SolrEntityProcessor.class); + + public static final String SOLR_SERVER = "url"; + public static final String QUERY = "query"; + /** + * (format="javabin|xml") default is javabin + */ + public static final String FORMAT = "format"; + public static final String ROWS = "rows"; + public static final String FIELDS = "fields"; + public static final String FQ = "fq"; + public static final String TIMEOUT = "timeout"; + + public static final int TIMEOUT_SECS = 5 * 60; // 5 minutes + public static final int ROWS_DEFAULT = 50; + + private SolrServer solrServer = null; + private String queryString; + private int rows = ROWS_DEFAULT; + private String[] filterQueries; + private String[] fields; + private int timeout = TIMEOUT_SECS; + + private boolean initDone = false; + + @Override + protected void firstInit(Context context) { + super.firstInit(context); + + try { + String serverPath = context.getResolvedEntityAttribute(SOLR_SERVER); + if (serverPath == null) { + throw new DataImportHandlerException(DataImportHandlerException.SEVERE, + "SolrEntityProcessor: parameter 'url' is required"); + } + HttpClient client = new HttpClient( + new MultiThreadedHttpConnectionManager()); + URL url = new URL(serverPath); + + if ("xml".equals(context.getResolvedEntityAttribute(FORMAT))) { + solrServer = new CommonsHttpSolrServer(url, client, + new XMLResponseParser(), false); + LOG.info("using XMLResponseParser"); + } else { + solrServer = new CommonsHttpSolrServer(url, client); + LOG.info("using BinaryResponseParser"); + } + + } catch (MalformedURLException e) { + throw new DataImportHandlerException(DataImportHandlerException.SEVERE, e); + } + + this.queryString = context.getResolvedEntityAttribute(QUERY); + if (this.queryString == null) { + throw new DataImportHandlerException( + DataImportHandlerException.SEVERE, + "SolrEntityProcessor: parameter 'query' is required" + ); + } + + String rowsP = context.getResolvedEntityAttribute(ROWS); + if (rowsP != null) { + rows = Integer.parseInt(rowsP); + } + + String fqAsString = context.getResolvedEntityAttribute(FQ); + if (fqAsString != null) { + this.filterQueries = fqAsString.split(","); + } + + String fieldsAsString = context.getResolvedEntityAttribute(FIELDS); + if (fieldsAsString != null) { + this.fields = fieldsAsString.split(","); + } + + String timeoutAsString = context.getResolvedEntityAttribute(TIMEOUT); + if (timeoutAsString != null) { + this.timeout = Integer.parseInt(timeoutAsString); + } + } + + @Override + public Map nextRow() { + buildIterator(); + return getNext(); + } + + /** + * The following method changes the rowIterator mutable field. It requires + * external synchronization. In fact when used in a multi-threaded setup the nextRow() method is called from a + * synchronized block {@link ThreadedEntityProcessorWrapper#nextRow()}, so this + * is taken care of. + */ + private void buildIterator() { + if (rowIterator == null) { + // We could use an AtomicBoolean but there's no need since this method + // would require anyway external synchronization + if (!initDone) { + initDone = true; + SolrDocumentList solrDocumentList = doQuery(0); + if (solrDocumentList != null) { + rowIterator = new SolrDocumentListIterator(solrDocumentList); + } + } + return; + } + + SolrDocumentListIterator documentListIterator = (SolrDocumentListIterator) rowIterator; + if (!documentListIterator.hasNext() && documentListIterator.hasMoreRows()) { + SolrDocumentList solrDocumentList = doQuery(documentListIterator + .getStart() + documentListIterator.getSize()); + if (solrDocumentList != null) { + rowIterator = new SolrDocumentListIterator(solrDocumentList); + } + } + + } + + protected SolrDocumentList doQuery(int start) { + SolrQuery solrQuery = new SolrQuery(queryString); + solrQuery.setRows(rows); + solrQuery.setStart(start); + if (fields != null) { + for (String field : fields) { + solrQuery.addField(field); + } + } + solrQuery.setFilterQueries(filterQueries); + solrQuery.setTimeAllowed(timeout * 1000); + + QueryResponse response = null; + try { + response = solrServer.query(solrQuery); + } catch (SolrServerException e) { + if (ABORT.equals(onError)) { + wrapAndThrow(SEVERE, e); + } else if (SKIP.equals(onError)) { + wrapAndThrow(DataImportHandlerException.SKIP_ROW, e); + } + } + + return response == null ? null : response.getResults(); + } + + private static class SolrDocumentListIterator implements Iterator> { + + private final int start; + private final int size; + private final long numFound; + private final Iterator solrDocumentIterator; + + public SolrDocumentListIterator(SolrDocumentList solrDocumentList) { + this.solrDocumentIterator = solrDocumentList.iterator(); + this.numFound = solrDocumentList.getNumFound(); + // SolrQuery has the start field of type int while SolrDocumentList of + // type long. We are always querying with an int so we can't receive a + // long as output. That's the reason why the following cast seems safe + this.start = (int) solrDocumentList.getStart(); + this.size = solrDocumentList.size(); + } + + @Override + public boolean hasNext() { + return solrDocumentIterator.hasNext(); + } + + @Override + public Map next() { + SolrDocument solrDocument = solrDocumentIterator.next(); + + HashMap map = new HashMap(); + Collection fields = solrDocument.getFieldNames(); + for (String field : fields) { + Object fieldValue = solrDocument.getFieldValue(field); + map.put(field, fieldValue); + } + return map; + } + + public int getStart() { + return start; + } + + public int getSize() { + return size; + } + + public boolean hasMoreRows() { + return numFound > start + size; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + } + +} diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java new file mode 100644 index 00000000000..675fff017aa --- /dev/null +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/MockSolrEntityProcessor.java @@ -0,0 +1,64 @@ +package org.apache.solr.handler.dataimport; + +/* + * 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. + */ + +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrDocumentList; + +public class MockSolrEntityProcessor extends SolrEntityProcessor { + + private final String[][][] docsData; + private final int rows; + private int queryCount = 0; + + public MockSolrEntityProcessor(String[][][] docsData) { + this(docsData, ROWS_DEFAULT); + } + + public MockSolrEntityProcessor(String[][][] docsData, int rows) { + this.docsData = docsData; + this.rows = rows; + } + + @Override + protected SolrDocumentList doQuery(int start) { + queryCount++; + return getDocs(start, rows); + } + + private SolrDocumentList getDocs(int start, int rows) { + SolrDocumentList docs = new SolrDocumentList(); + docs.setNumFound(docsData.length); + docs.setStart(start); + + int endIndex = start + rows; + int end = docsData.length < endIndex ? docsData.length : endIndex; + for (int i = start; i < end; i++) { + SolrDocument doc = new SolrDocument(); + for (String[] fields : docsData[i]) { + doc.addField(fields[0], fields[1]); + } + docs.add(doc); + } + return docs; + } + + public int getQueryCount() { + return queryCount; + } +} diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java new file mode 100644 index 00000000000..1dd0c9c2f01 --- /dev/null +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorEndToEnd.java @@ -0,0 +1,334 @@ +package org.apache.solr.handler.dataimport; + +/* + * 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. + */ + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.commons.httpclient.HttpClient; +import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager; +import org.apache.commons.io.FileUtils; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.embedded.JettySolrRunner; +import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer; +import org.apache.solr.common.SolrInputDocument; +import org.junit.After; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * End-to-end test of SolrEntityProcessor. "Real" test using embedded Solr + */ +public class TestSolrEntityProcessorEndToEnd extends AbstractDataImportHandlerTestCase { + + private static Logger LOG = LoggerFactory.getLogger(TestSolrEntityProcessorEndToEnd.class); + + private static final String SOLR_SOURCE_URL = "http://localhost:8983/solr"; + private static final String SOLR_CONFIG = "dataimport-solrconfig.xml"; + private static final String SOLR_SCHEMA = "dataimport-schema.xml"; + private static final String SOLR_HOME = "dih/solr"; + private static final String CONF_DIR = "dih" + File.separator + "solr" + File.separator + "conf" + File.separator; + + private static final List> DB_DOCS = new ArrayList>(); + private static final List> SOLR_DOCS = new ArrayList>(); + + static { + // dynamic fields in the destination schema + Map dbDoc = new HashMap(); + dbDoc.put("dbid_s", "1"); + dbDoc.put("dbdesc_s", "DbDescription"); + DB_DOCS.add(dbDoc); + + Map solrDoc = new HashMap(); + solrDoc.put("id", "1"); + solrDoc.put("desc", "SolrDescription"); + SOLR_DOCS.add(solrDoc); + } + + private static final String DIH_CONFIG_TAGS_INNER_ENTITY = "\r\n" + + " \r\n" + + " \r\n" + + " \r\n" + + " \r\n" + + " \r\n" + + " \r\n" + + " \r\n" + + " \r\n" + " \r\n" + + " \r\n" + " \r\n" + "\r\n"; + + private SolrInstance instance = null; + private JettySolrRunner jetty; + + private static String generateDIHConfig(String options) { + return "\r\n" + " \r\n" + + " \r\n" + " \r\n" + + "\r\n"; + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + // destination solr core + initCore(SOLR_CONFIG, SOLR_SCHEMA, SOLR_HOME); + // data source solr instance + instance = new SolrInstance(); + instance.setUp(); + jetty = createJetty(instance); + } + + @Override + @After + public void tearDown() throws Exception { + try { + deleteCore(); + } catch (Exception e) { + LOG.error("Error deleting core", e); + } + jetty.stop(); + instance.tearDown(); + super.tearDown(); + } + + public void testFullImport() { + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + addDocumentsToSolr(SOLR_DOCS); + runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'")); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='1']"); + assertQ(req("id:1"), "//result/doc/str[@name='id'][.='1']", + "//result/doc/arr[@name='desc'][.='SolrDescription']"); + } + + public void testFullImportFqParam() { + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + addDocumentsToSolr(generateSolrDocuments(30)); + Map map = new HashMap(); + map.put("rows", "50"); + runFullImport(generateDIHConfig("query='*:*' fq='desc:Description1*,desc:Description*2' rows='2'"), map); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='1']"); + assertQ(req("id:12"), "//result[@numFound='1']", "//result/doc/arr[@name='desc'][.='Description12']"); + } + + public void testFullImportFieldsParam() { + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + addDocumentsToSolr(generateSolrDocuments(7)); + runFullImport(generateDIHConfig("query='*:*' fields='id' rows='2'")); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='7']"); + assertQ(req("id:1"), "//result[@numFound='1']"); + try { + assertQ(req("id:1"), "//result/doc/arr[@name='desc']"); + fail("The document has a field with name desc"); + } catch(Exception e) { + + } + + } + + /** + * Receive a row from SQL (Mock) and fetch a row from Solr + */ + public void testFullImportInnerEntity() { + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + MockDataSource.setIterator("select * from x", DB_DOCS.iterator()); + addDocumentsToSolr(SOLR_DOCS); + runFullImport(DIH_CONFIG_TAGS_INNER_ENTITY); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } finally { + MockDataSource.clearCache(); + } + + assertQ(req("*:*"), "//result[@numFound='1']"); + assertQ(req("id:1"), "//result/doc/str[@name='id'][.='1']", + "//result/doc/str[@name='dbdesc_s'][.='DbDescription']", + "//result/doc/str[@name='dbid_s'][.='1']", + "//result/doc/arr[@name='desc'][.='SolrDescription']"); + + } + + public void testFullImportWrongSolrUrl() { + try { + jetty.stop(); + } catch (Exception e) { + LOG.error("Error stopping jetty", e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + runFullImport(generateDIHConfig("query='*:*' rows='2' fields='id,desc' onError='skip'")); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='0']"); + } + + public void testFullImportBadConfig() { + assertQ(req("*:*"), "//result[@numFound='0']"); + + try { + runFullImport(generateDIHConfig("query='bogus:3' rows='2' fields='id,desc' onError='abort'")); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='0']"); + } + + public void testFullImportMultiThreaded() { + assertQ(req("*:*"), "//result[@numFound='0']"); + int numDocs = 37; + List> docList = generateSolrDocuments(numDocs); + + try { + addDocumentsToSolr(docList); + Map map = new HashMap(); + map.put("rows", "50"); + runFullImport(generateDIHConfig("query='*:*' rows='6' numThreads='4'"), + map); + } catch (Exception e) { + LOG.error(e.getMessage(), e); + fail(e.getMessage()); + } + + assertQ(req("*:*"), "//result[@numFound='" + numDocs + "']"); + } + + private static List> generateSolrDocuments(int num) { + List> docList = new ArrayList>(); + for (int i = 1; i <= num; i++) { + Map map = new HashMap(); + map.put("id", i); + map.put("desc", "Description" + i); + docList.add(map); + } + return docList; + } + + private void addDocumentsToSolr(List> docs) throws SolrServerException, IOException { + List sidl = new ArrayList(); + for (Map doc : docs) { + SolrInputDocument sd = new SolrInputDocument(); + for (Entry entry : doc.entrySet()) { + sd.addField(entry.getKey(), entry.getValue()); + } + sidl.add(sd); + } + + HttpClient client = new HttpClient(new MultiThreadedHttpConnectionManager()); + URL url = new URL(SOLR_SOURCE_URL); + CommonsHttpSolrServer solrServer = new CommonsHttpSolrServer(url, client); + solrServer.add(sidl); + solrServer.commit(true, true); + } + + private static class SolrInstance { + + File homeDir; + File confDir; + + public String getHomeDir() { + return homeDir.toString(); + } + + public String getSchemaFile() { + return CONF_DIR + "dataimport-schema.xml"; + } + + public String getDataDir() { + return dataDir.toString(); + } + + public String getSolrConfigFile() { + return CONF_DIR + "dataimport-solrconfig.xml"; + } + + public void setUp() throws Exception { + + File home = new File(TEMP_DIR, getClass().getName() + "-" + + System.currentTimeMillis()); + + homeDir = new File(home + "inst"); + dataDir = new File(homeDir, "data"); + confDir = new File(homeDir, "conf"); + + homeDir.mkdirs(); + dataDir.mkdirs(); + confDir.mkdirs(); + + File f = new File(confDir, "solrconfig.xml"); + FileUtils.copyFile(getFile(getSolrConfigFile()), f); + f = new File(confDir, "schema.xml"); + + FileUtils.copyFile(getFile(getSchemaFile()), f); + f = new File(confDir, "data-config.xml"); + FileUtils.copyFile(getFile(CONF_DIR + "dataconfig-contentstream.xml"), f); + } + + public void tearDown() throws Exception { + recurseDelete(homeDir); + } + + } + + private JettySolrRunner createJetty(SolrInstance instance) throws Exception { + System.setProperty("solr.solr.home", instance.getHomeDir()); + System.setProperty("solr.data.dir", instance.getDataDir()); + JettySolrRunner jetty = new JettySolrRunner("/solr", 8983); + jetty.start(); + return jetty; + } + +} diff --git a/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java new file mode 100644 index 00000000000..57423e91038 --- /dev/null +++ b/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSolrEntityProcessorUnit.java @@ -0,0 +1,149 @@ +/** + * 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.handler.dataimport; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Unit test of SolrEntityProcessor. A very basic test outside of the DIH. + */ +public class TestSolrEntityProcessorUnit extends AbstractDataImportHandlerTestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestSolrEntityProcessorUnit.class); + private static final String ID = "id"; + + public void testQuery() { + String[][][] docs = generateDocs(2); + + MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs); + + assertExpectedDocs(docs, processor); + assertEquals(1, processor.getQueryCount()); + } + + public void testNumDocsGreaterThanRows() { + String[][][] docs = generateDocs(44); + + MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs, 10); + assertExpectedDocs(docs, processor); + assertEquals(5, processor.getQueryCount()); + } + + public void testMultiValuedFields() { + String[][][] docs = new String[1][2][2]; + String[][] doc = new String[][] { {"id", "1"}, {"multi", "multi1"}, + {"multi", "multi2"}, {"multi", "multi3"}}; + docs[0] = doc; + + MockSolrEntityProcessor processor = new MockSolrEntityProcessor(docs); + + Map next = processor.nextRow(); + assertNotNull(next); + assertEquals(doc[0][1], next.get(doc[0][0])); + + String[] multiValued = {"multi1", "multi2", "multi3"}; + assertEquals(Arrays.asList(multiValued), next.get(doc[1][0])); + assertEquals(1, processor.getQueryCount()); + assertNull(processor.nextRow()); + + } + + public void testMultiThread() { + int numThreads = 5; + int numDocs = 40; + String[][][] docs = generateDocs(numDocs); + final MockSolrEntityProcessor entityProcessor = new MockSolrEntityProcessor(docs, 25); + + final Map> rowList = new HashMap>(); + final CountDownLatch latch = new CountDownLatch(numThreads); + for (int i = 0; i < numThreads; i++) { + Runnable runnable = new Runnable() { + public void run() { + try { + while (true) { + Map row; + synchronized (entityProcessor) { + row = entityProcessor.nextRow(); + } + if (row == null) { + break; + } + rowList.put(row.get(ID).toString(), row); + } + } finally { + latch.countDown(); + } + } + }; + + new ThreadPoolExecutor(0, Integer.MAX_VALUE, 5, TimeUnit.SECONDS, + new SynchronousQueue()).execute(runnable); + } + + try { + latch.await(); + } catch (InterruptedException e) { + LOG.error(e.getMessage(), e); + } + + assertEquals(numDocs, rowList.size()); + + for (String[][] expectedDoc : docs) { + Map row = rowList.get(expectedDoc[0][1]); + assertNotNull(row); + int i = 0; + for (Entry entry : row.entrySet()) { + assertEquals(expectedDoc[i][0], entry.getKey()); + assertEquals(expectedDoc[i][1], entry.getValue()); + i++; + } + rowList.remove(expectedDoc[0][1]); + } + + assertEquals(0, rowList.size()); + + } + + private static String[][][] generateDocs(int numDocs) { + String[][][] docs = new String[numDocs][2][2]; + for (int i = 0; i < numDocs; i++) { + docs[i] = new String[][] { {"id", Integer.toString(i+1)}, + {"description", "Description" + Integer.toString(i+1)}}; + } + return docs; + } + + private static void assertExpectedDocs(String[][][] expectedDocs, SolrEntityProcessor processor) { + for (String[][] expectedDoc : expectedDocs) { + Map next = processor.nextRow(); + assertNotNull(next); + assertEquals(expectedDoc[0][1], next.get(expectedDoc[0][0])); + assertEquals(expectedDoc[1][1], next.get(expectedDoc[1][0])); + } + assertNull(processor.nextRow()); + } +} diff --git a/solr/example/example-DIH/README.txt b/solr/example/example-DIH/README.txt index b38b6d5705e..9c2f3c37e24 100644 --- a/solr/example/example-DIH/README.txt +++ b/solr/example/example-DIH/README.txt @@ -38,6 +38,10 @@ To import data from your imap server 1. Edit the example-DIH/solr/mail/conf/data-config.xml and add details about username, password, imap server 2. Connect to http://localhost:8983/solr/mail/dataimport?command=full-import +To copy data from db Solr core, connect to + + http://localhost:8983/solr/solr/dataimport?command=full-import + See also README.txt in the solr subdirectory, and check http://wiki.apache.org/solr/DataImportHandler for detailed usage guide and tutorial. diff --git a/solr/example/example-DIH/solr/solr.xml b/solr/example/example-DIH/solr/solr.xml index 39defb1f00d..802b9b32e09 100644 --- a/solr/example/example-DIH/solr/solr.xml +++ b/solr/example/example-DIH/solr/solr.xml @@ -5,5 +5,6 @@ + diff --git a/solr/example/example-DIH/solr/solr/conf/admin-extra.html b/solr/example/example-DIH/solr/solr/conf/admin-extra.html new file mode 100644 index 00000000000..aa739da862c --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/admin-extra.html @@ -0,0 +1,31 @@ + + + diff --git a/solr/example/example-DIH/solr/solr/conf/elevate.xml b/solr/example/example-DIH/solr/solr/conf/elevate.xml new file mode 100644 index 00000000000..7630ebe20fb --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/elevate.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/solr/conf/protwords.txt b/solr/example/example-DIH/solr/solr/conf/protwords.txt new file mode 100644 index 00000000000..1dfc0abecbf --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/protwords.txt @@ -0,0 +1,21 @@ +# 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. + +#----------------------------------------------------------------------- +# Use a protected word file to protect against the stemmer reducing two +# unrelated words to the same base word. + +# Some non-words that normally won't be encountered, +# just to test that they won't be stemmed. +dontstems +zwhacky + diff --git a/solr/example/example-DIH/solr/solr/conf/schema.xml b/solr/example/example-DIH/solr/solr/conf/schema.xml new file mode 100644 index 00000000000..678674fc63d --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/schema.xml @@ -0,0 +1,359 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + id + + + text + + + + + + + + + + + + + + + + + + + + + diff --git a/solr/example/example-DIH/solr/solr/conf/scripts.conf b/solr/example/example-DIH/solr/solr/conf/scripts.conf new file mode 100644 index 00000000000..f58b262ae0c --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/scripts.conf @@ -0,0 +1,24 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +user= +solr_hostname=localhost +solr_port=8983 +rsyncd_port=18983 +data_dir= +webapp_name=solr +master_host= +master_data_dir= +master_status_dir= diff --git a/solr/example/example-DIH/solr/solr/conf/solr-data-config.xml b/solr/example/example-DIH/solr/solr/conf/solr-data-config.xml new file mode 100644 index 00000000000..49873f55eb7 --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/solr-data-config.xml @@ -0,0 +1,22 @@ + + + + + + + diff --git a/solr/example/example-DIH/solr/solr/conf/solrconfig.xml b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml new file mode 100644 index 00000000000..d7c5f4eac2b --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/solrconfig.xml @@ -0,0 +1,569 @@ + + + + + + + LUCENE_40 + + + + + + + + false + + 10 + + + + 32 + 2147483647 + 10000 + 1000 + + + + + + + + + single + + + + + false + 32 + 10 + + + 2147483647 + 10000 + + + false + + + + + + + + + 100000 + + + + + + + + + + + + + 1024 + + + + + + + + + + + + + true + + + + + + + + 50 + + + 200 + + + + + + + + + solr 0 10 + rocks 0 10 + static newSearcher warming query from solrconfig.xml + + + + + + + + + + + false + + + 4 + + + + + + + + + + + + + + + + + + + + + + + explicit + + + + + + + + + 1 + 0.5 + + + + + + + + spell + + + + + word + + + + + + manu,cat + 1 + + + + + + solr-data-config.xml + + + + + + + + explicit + + + + + + + string + elevate.xml + + + + + explicit + + + elevator + + + + + + + + + + + + + + + + + + + + + explicit + true + + + + + + + + + 100 + + + + + + + + 70 + + 0.5 + + [-\w ,/\n\"']{20,200} + + + + + + + ]]> + ]]> + + + + + + + + + + 5 + + + + + *:* + + + + + + diff --git a/solr/example/example-DIH/solr/solr/conf/stopwords.txt b/solr/example/example-DIH/solr/solr/conf/stopwords.txt new file mode 100644 index 00000000000..b5824da3263 --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/stopwords.txt @@ -0,0 +1,58 @@ +# 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. + +#----------------------------------------------------------------------- +# a couple of test stopwords to test that the words are really being +# configured from this file: +stopworda +stopwordb + +#Standard english stop words taken from Lucene's StopAnalyzer +a +an +and +are +as +at +be +but +by +for +if +in +into +is +it +no +not +of +on +or +s +such +t +that +the +their +then +there +these +they +this +to +was +will +with + diff --git a/solr/example/example-DIH/solr/solr/conf/synonyms.txt b/solr/example/example-DIH/solr/solr/conf/synonyms.txt new file mode 100644 index 00000000000..b0e31cb7ec8 --- /dev/null +++ b/solr/example/example-DIH/solr/solr/conf/synonyms.txt @@ -0,0 +1,31 @@ +# 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. + +#----------------------------------------------------------------------- +#some test synonym mappings unlikely to appear in real input text +aaa => aaaa +bbb => bbbb1 bbbb2 +ccc => cccc1,cccc2 +a\=>a => b\=>b +a\,a => b\,b +fooaaa,baraaa,bazaaa + +# Some synonym groups specific to this example +GB,gib,gigabyte,gigabytes +MB,mib,megabyte,megabytes +Television, Televisions, TV, TVs +#notice we use "gib" instead of "GiB" so any WordDelimiterFilter coming +#after us won't split it into two words. + +# Synonym mappings can be used for spelling correction too +pixima => pixma +