Removed superfluous tests. Tweaked testing parameters.
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_inverted_index_updates_new@1879 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
index a22a989..a98e6f3 100644
--- a/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
+++ b/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/config/AccessMethodTestsConfig.java
@@ -74,12 +74,12 @@
public static final int LSM_INVINDEX_DISK_MAX_OPEN_FILES = 1000;
public static final int LSM_INVINDEX_MEM_PAGE_SIZE = 1024;
public static final int LSM_INVINDEX_MEM_NUM_PAGES = 100;
- public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 256;
+ public static final int LSM_INVINDEX_HYRACKS_FRAME_SIZE = 32768;
// Test parameters.
- public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 100;
+ public static final int LSM_INVINDEX_NUM_DOCS_TO_INSERT = 10000;
// Used for full-fledged search test.
- public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 1000;
- public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 1000;
+ public static final int LSM_INVINDEX_NUM_DOC_QUERIES = 500;
+ public static final int LSM_INVINDEX_NUM_RANDOM_QUERIES = 500;
// Used for non-search tests to sanity check index searches.
public static final int LSM_INVINDEX_TINY_NUM_DOC_QUERIES = 200;
public static final int LSM_INVINDEX_TINY_NUM_RANDOM_QUERIES = 200;
@@ -89,7 +89,7 @@
public static final int LSM_INVINDEX_NUM_DELETE_ROUNDS = 3;
// Allocate a generous size to make sure we have enough elements for all tests.
public static final int LSM_INVINDEX_SCAN_COUNT_ARRAY_SIZE = 1000000;
- public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 10000;
+ public static final int LSM_INVINDEX_MULTITHREAD_NUM_OPERATIONS = 5000;
}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexSearchTest.java
deleted file mode 100644
index bcfbb14..0000000
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexSearchTest.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.junit.After;
-import org.junit.Before;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
-import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.FixedSizeElementInvertedListBuilder;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
-import edu.uci.ics.hyracks.test.support.TestUtils;
-
-public abstract class AbstractInvIndexSearchTest extends AbstractInvIndexTest {
- protected final int PAGE_SIZE = 32768;
- protected final int NUM_PAGES = 100;
- protected final int MAX_OPEN_FILES = 10;
- protected final int HYRACKS_FRAME_SIZE = 32768;
- protected IHyracksTaskContext taskCtx = TestUtils.create(HYRACKS_FRAME_SIZE);
-
- protected IBufferCache bufferCache;
- protected IFileMapProvider fmp;
-
- protected FileReference invListsFile = new FileReference(new File(invListsFileName));
- protected FileReference btreeFile = new FileReference(new File(invListsFileName + "_btree"));
-
- protected ITypeTraits[] tokenTypeTraits = new ITypeTraits[] { UTF8StringPointable.TYPE_TRAITS };
- protected IBinaryComparatorFactory[] tokenCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) };
- protected ITypeTraits[] invListTypeTraits = new ITypeTraits[] { IntegerPointable.TYPE_TRAITS };
- protected IBinaryComparatorFactory[] invListCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(IntegerPointable.FACTORY) };
-
- protected OnDiskInvertedIndex invIndex;
-
- protected Random rnd = new Random();
-
- protected ArrayTupleBuilder tb = new ArrayTupleBuilder(2);
- protected ArrayTupleReference tuple = new ArrayTupleReference();
-
- protected ISerializerDeserializer[] insertSerde = { UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE };
- protected RecordDescriptor insertRecDesc = new RecordDescriptor(insertSerde);
-
- protected ArrayList<ArrayList<Integer>> checkInvLists = new ArrayList<ArrayList<Integer>>();
-
- protected int maxId = 1000000;
- protected int[] scanCountArray = new int[maxId];
- protected ArrayList<Integer> expectedResults = new ArrayList<Integer>();
-
- protected ISerializerDeserializer[] querySerde = { UTF8StringSerializerDeserializer.INSTANCE };
- protected RecordDescriptor queryRecDesc = new RecordDescriptor(querySerde);
-
- protected ArrayTupleBuilder queryTb = new ArrayTupleBuilder(querySerde.length);
- protected ArrayTupleReference queryTuple = new ArrayTupleReference();
-
- protected ITokenFactory tokenFactory;
- protected IBinaryTokenizer tokenizer;
-
- protected IIndexCursor resultCursor;
-
- protected abstract void setTokenizer();
-
- /**
- * Initialize members, generate data, and bulk load the inverted index.
- */
- @Before
- public void start() throws Exception {
- TestStorageManagerComponentHolder.init(PAGE_SIZE, NUM_PAGES, MAX_OPEN_FILES);
- bufferCache = TestStorageManagerComponentHolder.getBufferCache(taskCtx);
- fmp = TestStorageManagerComponentHolder.getFileMapProvider(taskCtx);
-
- setTokenizer();
-
- IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
- invIndex = new OnDiskInvertedIndex(bufferCache, fmp, invListBuilder, invListTypeTraits, invListCmpFactories,
- tokenTypeTraits, tokenCmpFactories, invListsFile, btreeFile);
- invIndex.create();
- invIndex.activate();
-
- rnd.setSeed(50);
- }
-
- @After
- public void deinit() throws HyracksDataException {
- invIndex.deactivate();
- invIndex.destroy();
- bufferCache.close();
- AbstractInvIndexTest.tearDown();
- }
-}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexTest.java
deleted file mode 100644
index 92f6bdc..0000000
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/AbstractInvIndexTest.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk;
-
-import java.io.File;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.logging.Logger;
-
-public abstract class AbstractInvIndexTest {
-
- protected static final Logger LOGGER = Logger.getLogger(AbstractInvIndexTest.class.getName());
-
- protected final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
- protected final static String tmpDir = System.getProperty("java.io.tmpdir");
- protected final static String sep = System.getProperty("file.separator");
- protected final static String baseFileName = tmpDir + sep + simpleDateFormat.format(new Date());
- protected final static String invListsFileName = baseFileName + "invlists";
-
- public static void tearDown() {
- File invListsFile = new File(invListsFileName);
- invListsFile.deleteOnExit();
- }
-}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchPerfTest.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchPerfTest.java
deleted file mode 100644
index ec0d305..0000000
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchPerfTest.java
+++ /dev/null
@@ -1,289 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.logging.Level;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex.OnDiskInvertedIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
-
-/**
- * The purpose of this test is to evaluate the performance of searches against
- * an inverted index. First, we generate random <token, id> pairs sorted on
- * token, which are bulk loaded into an inverted index. Next, we build random
- * queries from a list of predefined tokens in the index, and measure the
- * performance of executing them with different search modifiers. We test the
- * ConjunctiveSearchModifier and the JaccardSearchModifier.
- */
-public class OnDiskInvertedIndexSearchPerfTest extends AbstractInvIndexSearchTest {
-
- protected List<String> tokens = new ArrayList<String>();
-
- @Override
- protected void setTokenizer() {
- tokenFactory = new UTF8WordTokenFactory();
- tokenizer = new DelimitedUTF8StringBinaryTokenizer(true, false, tokenFactory);
- }
-
- @Before
- public void start() throws Exception {
- super.start();
- loadData();
- }
-
- public void loadData() throws HyracksDataException, IndexException {
- tokens.add("compilers");
- tokens.add("computer");
- tokens.add("databases");
- tokens.add("fast");
- tokens.add("hyracks");
- tokens.add("major");
- tokens.add("science");
- tokens.add("systems");
- tokens.add("university");
-
- for (int i = 0; i < tokens.size(); i++) {
- checkInvLists.add(new ArrayList<Integer>());
- }
-
- // for generating length-skewed inverted lists
- int addProb = 0;
- int addProbStep = 10;
-
- IIndexBulkLoader bulkLoader = invIndex.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false);
-
- for (int i = 0; i < tokens.size(); i++) {
-
- addProb += addProbStep * (i + 1);
- for (int j = 0; j < maxId; j++) {
- if ((Math.abs(rnd.nextInt()) % addProb) == 0) {
- tb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(tokens.get(i), tb.getDataOutput());
- tb.addFieldEndOffset();
- IntegerSerializerDeserializer.INSTANCE.serialize(j, tb.getDataOutput());
- tb.addFieldEndOffset();
- tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
- checkInvLists.get(i).add(j);
- try {
- bulkLoader.add(tuple);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- }
- }
- bulkLoader.end();
- }
-
- /**
- * Determine the expected results with the ScanCount algorithm. The
- * ScanCount algorithm is very simple, so we can be confident the results
- * are correct.
- */
- protected void fillExpectedResults(int[] queryTokenIndexes, int numQueryTokens, int occurrenceThreshold) {
- // reset scan count array
- for (int i = 0; i < maxId; i++) {
- scanCountArray[i] = 0;
- }
-
- // count occurrences
- for (int i = 0; i < numQueryTokens; i++) {
- ArrayList<Integer> list = checkInvLists.get(queryTokenIndexes[i]);
- for (int j = 0; j < list.size(); j++) {
- scanCountArray[list.get(j)]++;
- }
- }
-
- // check threshold
- expectedResults.clear();
- for (int i = 0; i < maxId; i++) {
- if (scanCountArray[i] >= occurrenceThreshold) {
- expectedResults.add(i);
- }
- }
- }
-
- /**
- * Generates a specified number of queries. Each query consists of a set of
- * randomly chosen tokens that are picked from the pre-defined set of
- * tokens. We run each query, measure it's time, and verify it's results
- * against the results produced by ScanCount, implemented in
- * fillExpectedResults().
- */
- private void runQueries(IInvertedIndexSearchModifier searchModifier, int numQueries) throws Exception {
- rnd.setSeed(50);
-
- OnDiskInvertedIndexAccessor accessor = (OnDiskInvertedIndexAccessor) invIndex.createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
-
- // generate random queries
- int[] queryTokenIndexes = new int[tokens.size()];
- for (int i = 0; i < numQueries; i++) {
-
- int numQueryTokens = Math.abs(rnd.nextInt() % tokens.size()) + 1;
- for (int j = 0; j < numQueryTokens; j++) {
- queryTokenIndexes[j] = Math.abs(rnd.nextInt() % tokens.size());
- }
-
- StringBuilder strBuilder = new StringBuilder();
- for (int j = 0; j < numQueryTokens; j++) {
- strBuilder.append(tokens.get(queryTokenIndexes[j]));
- if (j + 1 != numQueryTokens) {
- strBuilder.append(" ");
- }
- }
-
- String queryString = strBuilder.toString();
-
- // Serialize query.
- queryTb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, queryTb.getDataOutput());
- queryTb.addFieldEndOffset();
- queryTuple.reset(queryTb.getFieldEndOffsets(), queryTb.getByteArray());
-
- // Set query tuple in search predicate.
- searchPred.setQueryTuple(queryTuple);
- searchPred.setQueryFieldIndex(0);
-
- boolean panic = false;
-
- resultCursor = accessor.createSearchCursor();
- int repeats = 1;
- double totalTime = 0;
- for (int j = 0; j < repeats; j++) {
- long timeStart = System.currentTimeMillis();
- try {
- resultCursor.reset();
- accessor.search(resultCursor, searchPred);
- } catch (OccurrenceThresholdPanicException e) {
- panic = true;
- }
- long timeEnd = System.currentTimeMillis();
- totalTime += timeEnd - timeStart;
- }
- double avgTime = totalTime / (double) repeats;
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(i + ": " + "\"" + queryString + "\": " + avgTime + "ms");
- }
-
- if (!panic) {
- int occurrenceThreshold = searchModifier.getOccurrenceThreshold(numQueryTokens);
- fillExpectedResults(queryTokenIndexes, numQueryTokens, occurrenceThreshold);
- // verify results
- int checkIndex = 0;
- while (resultCursor.hasNext()) {
- resultCursor.next();
- ITupleReference resultTuple = resultCursor.getTuple();
- int id = IntegerSerializerDeserializer.getInt(resultTuple.getFieldData(0),
- resultTuple.getFieldStart(0));
- Assert.assertEquals(expectedResults.get(checkIndex).intValue(), id);
- checkIndex++;
- }
-
- if (expectedResults.size() != checkIndex) {
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("CHECKING");
- }
- StringBuilder expectedStrBuilder = new StringBuilder();
- for (Integer x : expectedResults) {
- expectedStrBuilder.append(x + " ");
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(expectedStrBuilder.toString());
- }
- }
-
- Assert.assertEquals(expectedResults.size(), checkIndex);
- }
- }
- }
-
- /**
- * Runs 50 random conjunctive search queries to test the
- * ConjunctiveSearchModifier.
- */
- @Test
- public void conjunctiveKeywordQueryTest() throws Exception {
- IInvertedIndexSearchModifier searchModifier = new ConjunctiveSearchModifier();
- runQueries(searchModifier, 50);
- }
-
- /**
- * Runs 50 random jaccard-based search queries with thresholds 1.0, 0.9,
- * 0.8, 0.7, 0.6, 0.5. Tests the JaccardSearchModifier.
- */
- @Test
- public void jaccardKeywordQueryTest() throws Exception {
- JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 1.0f);
- }
- searchModifier.setJaccThresh(1.0f);
- runQueries(searchModifier, 50);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.9f);
- }
- searchModifier.setJaccThresh(0.9f);
- runQueries(searchModifier, 50);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.8f);
- }
- searchModifier.setJaccThresh(0.8f);
- runQueries(searchModifier, 50);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.7f);
- }
- searchModifier.setJaccThresh(0.7f);
- runQueries(searchModifier, 50);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.6f);
- }
- searchModifier.setJaccThresh(0.6f);
- runQueries(searchModifier, 50);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.5f);
- }
- searchModifier.setJaccThresh(0.5f);
- runQueries(searchModifier, 50);
- }
-}
diff --git a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTestOld.java b/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTestOld.java
deleted file mode 100644
index f00747a..0000000
--- a/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexSearchTestOld.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed 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 from
- *
- * 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 edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.logging.Level;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
-import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.exceptions.OccurrenceThresholdPanicException;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.ondisk.OnDiskInvertedIndex.OnDiskInvertedIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.EditDistanceSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.InvertedIndexSearchPredicate;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifier;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
-
-public class OnDiskInvertedIndexSearchTestOld extends AbstractInvIndexSearchTest {
-
- protected List<String> dataStrings = new ArrayList<String>();
- protected List<String> firstNames = new ArrayList<String>();
- protected List<String> lastNames = new ArrayList<String>();
-
- protected IBinaryComparator[] btreeBinCmps;
-
- @Override
- protected void setTokenizer() {
- tokenFactory = new UTF8NGramTokenFactory();
- tokenizer = new NGramUTF8StringBinaryTokenizer(3, false, true, false, tokenFactory);
- }
-
- @Before
- public void start() throws Exception {
- super.start();
- btreeBinCmps = new IBinaryComparator[tokenCmpFactories.length];
- for (int i = 0; i < tokenCmpFactories.length; i++) {
- btreeBinCmps[i] = tokenCmpFactories[i].createBinaryComparator();
- }
- generateDataStrings();
- loadData();
- }
-
- public void generateDataStrings() {
- firstNames.add("Kathrin");
- firstNames.add("Cathrin");
- firstNames.add("Kathryn");
- firstNames.add("Cathryn");
- firstNames.add("Kathrine");
- firstNames.add("Cathrine");
- firstNames.add("Kathryne");
- firstNames.add("Cathryne");
- firstNames.add("Katherin");
- firstNames.add("Catherin");
- firstNames.add("Katheryn");
- firstNames.add("Catheryn");
- firstNames.add("Katherine");
- firstNames.add("Catherine");
- firstNames.add("Katheryne");
- firstNames.add("Catheryne");
- firstNames.add("John");
- firstNames.add("Jack");
- firstNames.add("Jonathan");
- firstNames.add("Nathan");
-
- lastNames.add("Miller");
- lastNames.add("Myller");
- lastNames.add("Keller");
- lastNames.add("Ketler");
- lastNames.add("Muller");
- lastNames.add("Fuller");
- lastNames.add("Smith");
- lastNames.add("Smyth");
- lastNames.add("Smithe");
- lastNames.add("Smythe");
-
- // Generate all 'firstName lastName' combinations as data strings
- for (String f : firstNames) {
- for (String l : lastNames) {
- dataStrings.add(f + " " + l);
- }
- }
- }
-
- private class TokenIdPair implements Comparable<TokenIdPair> {
- public ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
- public DataOutputStream dos = new DataOutputStream(baaos);
- public int id;
-
- TokenIdPair(IToken token, int id) throws IOException {
- token.serializeToken(dos);
- this.id = id;
- }
-
- @Override
- public int compareTo(TokenIdPair o) {
- int cmp = btreeBinCmps[0].compare(baaos.getByteArray(), 0, baaos.getByteArray().length,
- o.baaos.getByteArray(), 0, o.baaos.getByteArray().length);
- if (cmp == 0) {
- return id - o.id;
- } else {
- return cmp;
- }
- }
- }
-
- public void loadData() throws IOException, IndexException {
- List<TokenIdPair> pairs = new ArrayList<TokenIdPair>();
- // Generate pairs for subsequent sorting and bulk-loading.
- int id = 0;
- for (String s : dataStrings) {
- ByteArrayAccessibleOutputStream baaos = new ByteArrayAccessibleOutputStream();
- DataOutputStream dos = new DataOutputStream(baaos);
- UTF8StringSerializerDeserializer.INSTANCE.serialize(s, dos);
- tokenizer.reset(baaos.getByteArray(), 0, baaos.size());
- while (tokenizer.hasNext()) {
- tokenizer.next();
- IToken token = tokenizer.getToken();
- pairs.add(new TokenIdPair(token, id));
- }
- ++id;
- }
- Collections.sort(pairs);
-
- // Bulk load index.
- IIndexBulkLoader bulkLoader = invIndex.createBulkLoader(BTree.DEFAULT_FILL_FACTOR, false);
-
- for (TokenIdPair t : pairs) {
- tb.reset();
- tb.addField(t.baaos.getByteArray(), 0, t.baaos.getByteArray().length);
- IntegerSerializerDeserializer.INSTANCE.serialize(t.id, tb.getDataOutput());
- tb.addFieldEndOffset();
- tuple.reset(tb.getFieldEndOffsets(), tb.getByteArray());
-
- try {
- bulkLoader.add(tuple);
- } catch (Exception e) {
- e.printStackTrace();
- }
- }
- bulkLoader.end();
- }
-
- /**
- * Runs a specified number of randomly picked strings from dataStrings as
- * queries. We run each query, measure it's time, and print it's results.
- */
- private void runQueries(IInvertedIndexSearchModifier searchModifier, int numQueries) throws Exception {
- rnd.setSeed(50);
- OnDiskInvertedIndexAccessor accessor = (OnDiskInvertedIndexAccessor) invIndex.createAccessor(
- NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
- for (int i = 0; i < numQueries; i++) {
-
- int queryIndex = Math.abs(rnd.nextInt() % dataStrings.size());
- String queryString = dataStrings.get(queryIndex);
-
- // Serialize query.
- queryTb.reset();
- UTF8StringSerializerDeserializer.INSTANCE.serialize(queryString, queryTb.getDataOutput());
- queryTb.addFieldEndOffset();
- queryTuple.reset(queryTb.getFieldEndOffsets(), queryTb.getByteArray());
-
- // Set query tuple in search predicate.
- searchPred.setQueryTuple(queryTuple);
- searchPred.setQueryFieldIndex(0);
-
- resultCursor = accessor.createSearchCursor();
-
- int repeats = 1;
- double totalTime = 0;
- for (int j = 0; j < repeats; j++) {
- long timeStart = System.currentTimeMillis();
- try {
- resultCursor.reset();
- accessor.search(resultCursor, searchPred);
- } catch (OccurrenceThresholdPanicException e) {
- // ignore panic queries
- }
- long timeEnd = System.currentTimeMillis();
- totalTime += timeEnd - timeStart;
- }
- double avgTime = totalTime / (double) repeats;
- StringBuilder strBuilder = new StringBuilder();
- strBuilder.append(i + ": " + "\"" + queryString + "\": " + avgTime + "ms" + "\n");
- strBuilder.append("CANDIDATE RESULTS:\n");
- while (resultCursor.hasNext()) {
- resultCursor.next();
- ITupleReference resultTuple = resultCursor.getTuple();
- int id = IntegerSerializerDeserializer
- .getInt(resultTuple.getFieldData(0), resultTuple.getFieldStart(0));
- strBuilder.append(id + " " + dataStrings.get(id));
- strBuilder.append('\n');
- }
- // remove trailing newline
- strBuilder.deleteCharAt(strBuilder.length() - 1);
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(strBuilder.toString());
- }
- }
- }
-
- /**
- * Runs 5 random conjunctive search queries to test the
- * ConjunctiveSearchModifier.
- */
- @Test
- public void conjunctiveQueryTest() throws Exception {
- IInvertedIndexSearchModifier searchModifier = new ConjunctiveSearchModifier();
- runQueries(searchModifier, 5);
- }
-
- /**
- * Runs 5 random jaccard-based search queries with thresholds 0.9, 0.8, 0.7.
- * Tests the JaccardSearchModifier.
- */
- @Test
- public void jaccardQueryTest() throws Exception {
- JaccardSearchModifier searchModifier = new JaccardSearchModifier(1.0f);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.9f);
- }
- searchModifier.setJaccThresh(0.9f);
- runQueries(searchModifier, 5);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.8f);
- }
- searchModifier.setJaccThresh(0.8f);
- runQueries(searchModifier, 5);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("JACCARD: " + 0.7f);
- }
- searchModifier.setJaccThresh(0.7f);
- runQueries(searchModifier, 5);
- }
-
- /**
- * Runs 5 random edit-distance based search queries with thresholds 1, 2, 3.
- * Tests the EditDistanceSearchModifier.
- */
- @Test
- public void editDistanceQueryTest() throws Exception {
- EditDistanceSearchModifier searchModifier = new EditDistanceSearchModifier(3, 0);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("EDIT DISTANCE: " + 1);
- }
- searchModifier.setEdThresh(1);
- runQueries(searchModifier, 5);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("EDIT DISTANCE: " + 2);
- }
- searchModifier.setEdThresh(2);
- runQueries(searchModifier, 5);
-
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("EDIT DISTANCE: " + 3);
- }
- searchModifier.setEdThresh(3);
- runQueries(searchModifier, 5);
- }
-}