You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tika.apache.org by ju...@apache.org on 2009/11/06 23:53:07 UTC

svn commit: r833594 - in /lucene/tika/trunk/tika-core/src: main/java/org/apache/tika/language/ main/java/org/apache/tika/parser/ main/resources/org/apache/tika/language/ test/java/org/apache/tika/language/ test/resources/org/apache/tika/language/

Author: jukka
Date: Fri Nov  6 22:53:04 2009
New Revision: 833594

URL: http://svn.apache.org/viewvc?rev=833594&view=rev
Log:
TIKA-209: Language detection is weak

Refactor and simplify the language identifier code to better match the needs of Tika.

Integrate language detection to AutoDetectParser.

TODO: Javadocs

Added:
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageProfile.java
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingHandler.java
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingWriter.java
    lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageIdentifierTest.java
      - copied, changed from r829668, lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestLanguageIdentifier.java
    lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageProfileTest.java
Removed:
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/NGramEntry.java
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/NGramProfile.java
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/package.html
    lucene/tika/trunk/tika-core/src/main/resources/org/apache/tika/language/langmappings.properties
    lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestLanguageIdentifier.java
    lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestNGramProfile.java
    lucene/tika/trunk/tika-core/src/test/resources/org/apache/tika/language/test-referencial.txt
Modified:
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageIdentifier.java
    lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/parser/AutoDetectParser.java
    lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/ProfilingWriterTest.java

Modified: lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageIdentifier.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageIdentifier.java?rev=833594&r1=833593&r2=833594&view=diff
==============================================================================
--- lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageIdentifier.java (original)
+++ lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageIdentifier.java Fri Nov  6 22:53:04 2009
@@ -17,357 +17,113 @@
 package org.apache.tika.language;
 
 import java.io.BufferedReader;
-import java.io.ByteArrayOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.util.ArrayList;
-import java.util.Enumeration;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Properties;
-import java.util.Vector;
+import java.util.Map;
 
 /**
  * Identify the language of a content, based on statistical analysis.
+ * Text document language identifier.
+ * <p>
+ * Language profiles are based on material from
+ * <a href="http://www.isi.edu/~koehn/europarl/">http://www.isi.edu/~koehn/europarl/</a>.
  *
- * @see <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">ISO 639
- *      Language Codes</a>
- * 
- * @author Sami Siren
- * @author J&eacute;r&ocirc;me Charron
+ * @see <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">
+ *      ISO 639 Language Codes</a>
  */
 public class LanguageIdentifier {
 
-    private final static int DEFAULT_ANALYSIS_LENGTH = 0;    // 0 means full content
-
-    private ArrayList<NGramProfile> languages = new ArrayList<NGramProfile>();
-
-    private ArrayList<String> supportedLanguages = new ArrayList<String>();
-
-    /** Minimum size of NGrams */
-    private int minLength = NGramProfile.DEFAULT_MIN_NGRAM_LENGTH;
-
-    /** Maximum size of NGrams */
-    private int maxLength = NGramProfile.DEFAULT_MAX_NGRAM_LENGTH;
-
-    /** The maximum amount of data to analyze */
-    private int analyzeLength = DEFAULT_ANALYSIS_LENGTH;
-
-    /** A global index of ngrams of all supported languages */
-    private HashMap<CharSequence, NGramEntry[]> ngramsIdx = new HashMap<CharSequence, NGramEntry[]>();
-
-    /** The NGramProfile used for identification */
-    private NGramProfile suspect = null;
-
     /**
-     * Constructs a new Language Identifier.
+     * The available language profiles.
      */
-    public LanguageIdentifier(NGramProfile suspect) {
-        this.suspect = suspect;
+    private static final Map<String, LanguageProfile> profiles =
+        new HashMap<String, LanguageProfile>();
 
-        Properties p = new Properties();
+    private static void addProfile(String language) {
         try {
-            p.load(this.getClass().getResourceAsStream("langmappings.properties"));
-
-            Enumeration alllanguages = p.keys();
+            LanguageProfile profile = new LanguageProfile();
 
-            StringBuffer list = new StringBuffer("Language identifier plugin supports:");
-            HashMap<NGramEntry, List<NGramEntry>> tmpIdx = new HashMap<NGramEntry, List<NGramEntry>>();
-            while (alllanguages.hasMoreElements()) {
-                String lang = (String) (alllanguages.nextElement());
-
-                InputStream is = this.getClass().getClassLoader().getResourceAsStream(
-                        "org/apache/tika/language/" + lang + "." + NGramProfile.FILE_EXTENSION);
-
-                if (is != null) {
-                    NGramProfile profile = new NGramProfile(lang, minLength, maxLength);
-                    try {
-                        profile.load(is);
-                        languages.add(profile);
-                        supportedLanguages.add(lang);
-                        List<NGramEntry> ngrams = profile.getSorted();
-                        for (int i=0; i<ngrams.size(); i++) {
-                            NGramEntry entry = ngrams.get(i);
-                            List<NGramEntry> registered = tmpIdx.get(entry);
-                            if (registered == null) {
-                                registered = new ArrayList<NGramEntry>();
-                                tmpIdx.put(entry, registered);
-                            }
-                            registered.add(entry);
-                        }
-                        list.append(" " + lang + "(" + ngrams.size() + ")");
-                        is.close();
-                    } catch (IOException e1) {
-                        // if (LOG.isFatalEnabled()) { LOG.fatal(e1.toString()); }
+            InputStream stream =
+                LanguageIdentifier.class.getResourceAsStream(language + ".ngp");
+            try {
+                BufferedReader reader =
+                    new BufferedReader(new InputStreamReader(stream, "UTF-8"));
+                String line = reader.readLine();
+                while (line != null) {
+                    if (line.length() > 0 && !line.startsWith("#")) {
+                        int space = line.indexOf(' ');
+                        profile.add(
+                                line.substring(0, space),
+                                Long.parseLong(line.substring(space + 1)));
                     }
+                    line = reader.readLine();
                 }
+            } finally {
+                stream.close();
             }
-            // transform all ngrams lists to arrays for performances
-            Iterator<NGramEntry> keys = tmpIdx.keySet().iterator();
-            while (keys.hasNext()) {
-                NGramEntry entry = keys.next();
-                List<NGramEntry> l = tmpIdx.get(entry);
-                if (l != null) {
-                    NGramEntry[] array = l.toArray(new NGramEntry[l.size()]);
-                    ngramsIdx.put(entry.getSeq(), array);
-                }
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            // if (LOG.isFatalEnabled()) { LOG.fatal(e.toString()); }
+
+            profiles.put(language, profile);
+        } catch (Throwable t) {
+            // Failed to load this language profile. Log the problem?
         }
     }
 
-    public LanguageIdentifier() {
-        this(new NGramProfile(
-                "suspect",
-                NGramProfile.DEFAULT_MIN_NGRAM_LENGTH,
-                NGramProfile.DEFAULT_MAX_NGRAM_LENGTH));
+    static {
+        addProfile("da"); // Danish
+        addProfile("de"); // German
+        addProfile("ee");
+        addProfile("el"); // Greek
+        addProfile("en"); // English
+        addProfile("es"); // Spanish
+        addProfile("fi"); // Finnish
+        addProfile("fr"); // French
+        addProfile("hu"); // Hungarian
+        addProfile("is"); // Icelandic
+        addProfile("it"); // Italian
+        addProfile("nl"); // Dutch
+        addProfile("no"); // Norwegian
+        addProfile("pl"); // Polish
+        addProfile("pt"); // Portuguese
+        addProfile("ru"); // Russian
+        addProfile("sv"); // Swedish
+        addProfile("th"); // Thai
     }
 
-    /**
-     * Main method used for command line process.
-     * <br/>Usage is:
-     * <pre>
-     * LanguageIdentifier [-identifyrows filename maxlines]
-     *                    [-identifyfile charset filename]
-     *                    [-identifyfileset charset files]
-     *                    [-identifytext text]
-     *                    [-identifyurl url]
-     * </pre>
-     * @param args arguments.
-     */
-    public static void main(String args[]) {
-        String usage = "Usage: LanguageIdentifier" 
-            + " [-identifyrows filename maxlines]"
-            + " [-identifyfile charset filename]"
-            + " [-identifyfileset charset files]"
-            + " [-identifytext text] ";
-        int command = 0;
-
-        final int IDFILE = 1;
-        final int IDTEXT = 2;
-        final int IDFILESET = 4;
-        final int IDROWS = 5;
-
-        Vector<String> fileset = new Vector<String>();
-        String filename = "";
-        String charset = "";
-        String text = "";
-        int max = 0;
-
-        if (args.length == 0) {
-            System.err.println(usage);
-            System.exit(-1);
-        }
-
-        for (int i = 0; i < args.length; i++) { // parse command line
-            if (args[i].equals("-identifyfile")) {
-                command = IDFILE;
-                charset = args[++i];
-                filename = args[++i];
-            }
-
-            if (args[i].equals("-identifyrows")) {
-                command = IDROWS;
-                filename = args[++i];
-                max = Integer.parseInt(args[++i]);
-            }
-
-            if (args[i].equals("-identifytext")) {
-                command = IDTEXT;
-                for (i++; i < args.length - 1; i++)
-                    text += args[i] + " ";
-            }
-
-            if (args[i].equals("-identifyfileset")) {
-                command = IDFILESET;
-                charset = args[++i];
-                for (i++; i < args.length; i++) {
-                    File[] files = null;
-                    File f = new File(args[i]);
-                    if (f.isDirectory()) {
-                        files = f.listFiles();
-                    } else {
-                        files = new File[] { f };
-                    }
-                    for (int j=0; j<files.length; j++) {
-                        fileset.add(files[j].getAbsolutePath());
-                    }
-                }
-            }
-
-        }
+    private final String language;
 
-        String lang = null;
-        //LanguageIdentifier idfr = LanguageIdentifier.getInstance();
-        LanguageIdentifier idfr = new LanguageIdentifier();
-        File f;
-        FileInputStream fis;
-        try {
-            switch (command) {
+    private final double distance;
 
-            case IDTEXT:
-                lang = idfr.identify(text);
-                break;
-
-            case IDFILE:
-                f = new File(filename);
-                fis = new FileInputStream(f);
-                lang = idfr.identify(fis, charset);
-                fis.close();
-                break;
-
-            case IDROWS:
-                f = new File(filename);
-                BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(f)));
-                String line;
-                while (max > 0 && (line = br.readLine()) != null) {
-                    line = line.trim();
-                    if (line.length() > 2) {
-                        max--;
-                        lang = idfr.identify(line);
-                        System.out.println("R=" + lang + ":" + line);
-                    }
-                }
-
-                br.close();
-                System.exit(0);
-                break;
-
-            case IDFILESET:
-                System.out.println("FILESET");
-                Iterator<String> i = fileset.iterator();
-                while (i.hasNext()) {
-                    try {
-                        filename = i.next();
-                        f = new File(filename);
-                        fis = new FileInputStream(f);
-                        lang = idfr.identify(fis, charset);
-                        fis.close();
-                    } catch (Exception e) {
-                        System.out.println(e);
-                    }
-                    System.out.println(filename + " was identified as " + lang);
-                }
-                System.exit(0);
-                break;
+    public LanguageIdentifier(LanguageProfile profile) {
+        String minLanguage = "unknown";
+        double minDistance = 1.0;
+        for (Map.Entry<String, LanguageProfile> entry : profiles.entrySet()) {
+            double distance = profile.distance(entry.getValue());
+            if (distance < minDistance) {
+                minDistance = distance;
+                minLanguage = entry.getKey();
             }
-        } catch (Exception e) {
-            System.out.println(e);
         }
-        System.out.println("text was identified as " + lang);
-    }
 
-    /**
-     * Identify language of a content.
-     * 
-     * @param content is the content to analyze.
-     * @return The 2 letter
-     *         <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">ISO 639
-     *         language code</a> (en, fi, sv, ...) of the language that best
-     *         matches the specified content.
-     */
-    public String identify(String content) {
-        return identify(new StringBuilder(content));
+        this.language = minLanguage;
+        this.distance = minDistance;
     }
 
-    /**
-     * Identify language of a content.
-     * 
-     * @param content is the content to analyze.
-     * @return The 2 letter
-     *         <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">ISO 639
-     *         language code</a> (en, fi, sv, ...) of the language that best
-     *         matches the specified content.
-     */
-    public String identify(StringBuilder content) {
-        StringBuilder text = content;
-        if ((analyzeLength > 0) && (content.length() > analyzeLength)) {
-            text = new StringBuilder().append(content);
-            text.setLength(analyzeLength);
-        }
-        suspect.analyze(text);
-
-        return identify();
+    public LanguageIdentifier(String content) {
+        this(ProfilingWriter.profile(content));
     }
 
-    public String identify() {
-        Iterator<NGramEntry> iter = suspect.getSorted().iterator();
-        float topscore = Float.MIN_VALUE;
-        String lang = "";
-        HashMap<NGramProfile, Float> scores = new HashMap<NGramProfile, Float>();
-        NGramEntry searched = null;
-
-        while (iter.hasNext()) {
-            searched = iter.next();
-            NGramEntry[] ngrams = ngramsIdx.get(searched.getSeq());
-            if (ngrams != null) {
-                for (int j=0; j<ngrams.length; j++) {
-                    NGramProfile profile = ngrams[j].getProfile();
-                    Float pScore = scores.get(profile);
-                    if (pScore == null) {
-                        pScore = new Float(0);
-                    }
-                    float plScore = pScore.floatValue();
-                    plScore += ngrams[j].getFrequency() + searched.getFrequency();
-                    scores.put(profile, new Float(plScore));
-                    if (plScore > topscore) {
-                        topscore = plScore;
-                        lang = profile.getName();
-                    }
-                }
-            }
-        }
-        return lang;
+    public String getLanguage() {
+        return language;
     }
 
-    /**
-     * Identify language from input stream.
-     * This method uses the platform default encoding to read the input stream.
-     * For using a specific encoding, use the
-     * {@link #identify(InputStream, String)} method.
-     *
-     * @param is is the input stream to analyze.
-     * @return The 2 letter
-     *         <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">ISO 639
-     *         language code</a> (en, fi, sv, ...) of the language that best
-     *         matches the content of the specified input stream.
-     * @throws IOException if something wrong occurs on the input stream.
-     */
-    public String identify(InputStream is) throws IOException {
-        return identify(is, null);
+    public boolean isReasonablyCertain() {
+        return distance < 0.022;
     }
 
-    /**
-     * Identify language from input stream.
-     * 
-     * @param is is the input stream to analyze.
-     * @param charset is the charset to use to read the input stream.
-     * @return The 2 letter
-     *         <a href="http://www.w3.org/WAI/ER/IG/ert/iso639.htm">ISO 639
-     *         language code</a> (en, fi, sv, ...) of the language that best
-     *         matches the content of the specified input stream.
-     * @throws IOException if something wrong occurs on the input stream.
-     */
-    public String identify(InputStream is, String charset) throws IOException {
-        ByteArrayOutputStream out = new ByteArrayOutputStream();
-        byte[] buffer = new byte[2048];
-        int len = 0;
-
-        while (((len = is.read(buffer)) != -1) &&
-                ((analyzeLength == 0) || (out.size() < analyzeLength))) {
-            if (analyzeLength != 0) {
-                len = Math.min(len, analyzeLength - out.size());
-            }
-            out.write(buffer, 0, len);
-        }
-        return identify((charset == null) ? out.toString()
-                : out.toString(charset));
+    @Override
+    public String toString() {
+        return language + " (" + distance + ")";
     }
 
 }

Added: lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageProfile.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageProfile.java?rev=833594&view=auto
==============================================================================
--- lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageProfile.java (added)
+++ lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/LanguageProfile.java Fri Nov  6 22:53:04 2009
@@ -0,0 +1,116 @@
+/*
+ * 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.tika.language;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Language profile based on ngram counts.
+ */
+public class LanguageProfile {
+
+    /**
+     * The ngrams that make up this profile.
+     */
+    private final Map<String, Counter> ngrams =
+        new HashMap<String, Counter>();
+
+    /**
+     * The sum of all ngram counts in this profile.
+     * Used to calculate relative ngram frequency.
+     */
+    private long count = 0;
+
+    private class Counter {
+        private long count = 0;
+        public String toString() {
+            return Long.toString(count);
+        }
+    }
+
+    public long getCount() {
+        return count;
+    }
+
+    public long getCount(String ngram) {
+        Counter counter = ngrams.get(ngram);
+        if (counter != null) {
+            return counter.count;
+        } else {
+            return 0;
+        }
+    }
+
+    /**
+     * Adds a single occurrence of the given ngram to this profile.
+     *
+     * @param ngram the ngram
+     */
+    public void add(String ngram) {
+        add(ngram, 1);
+    }
+
+    /**
+     * Adds multiple occurrences of the given ngram to this profile.
+     *
+     * @param ngram the ngram
+     * @param count number of occurrences to add
+     */
+    public void add(String ngram, long count) {
+        Counter counter = ngrams.get(ngram);
+        if (counter == null) {
+            counter = new Counter();
+            ngrams.put(ngram, counter);
+        }
+        counter.count += count;
+        this.count += count;
+    }
+
+    /**
+     * Calculates the geometric distance between this and the given
+     * other language profile.
+     *
+     * @param that the other language profile
+     * @return distance between the profiles
+     */
+    public double distance(LanguageProfile that) {
+        double sumOfSquares = 0.0;
+        double thisCount = Math.max(this.count, 1.0);
+        double thatCount = Math.max(that.count, 1.0);
+
+        Set<String> ngrams = new HashSet<String>();
+        ngrams.addAll(this.ngrams.keySet());
+        ngrams.addAll(that.ngrams.keySet());
+        for (String ngram : ngrams) {
+            double thisFrequency = this.getCount(ngram) / thisCount;
+            double thatFrequency = that.getCount(ngram) / thatCount;
+            double difference = thisFrequency - thatFrequency;
+            sumOfSquares += difference * difference;
+        }
+
+        return Math.sqrt(sumOfSquares);
+    }
+
+    @Override
+    public String toString() {
+        return ngrams.toString();
+    }
+
+}

Added: lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingHandler.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingHandler.java?rev=833594&view=auto
==============================================================================
--- lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingHandler.java (added)
+++ lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingHandler.java Fri Nov  6 22:53:04 2009
@@ -0,0 +1,66 @@
+/*
+ * 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.tika.language;
+
+import org.apache.tika.metadata.Metadata;
+import org.apache.tika.sax.BodyContentHandler;
+import org.xml.sax.SAXException;
+
+public class ProfilingHandler extends BodyContentHandler {
+
+    private static final long CHECK_INTERVAL = 1000;
+
+    private final LanguageProfile profile;
+
+    private final Metadata metadata;
+
+    private long nextCheckCount = CHECK_INTERVAL;
+
+    private ProfilingHandler(ProfilingWriter writer, Metadata metadata) {
+        super(writer);
+        this.profile = writer.getProfile();
+        this.metadata = metadata;
+    }
+
+    public ProfilingHandler(Metadata metadata) {
+        this(new ProfilingWriter(), metadata);
+    }
+
+    private void checkAndSetLanguage() {
+        LanguageIdentifier identifier = new LanguageIdentifier(profile);
+        if (identifier.isReasonablyCertain()) {
+            metadata.set(Metadata.LANGUAGE, identifier.getLanguage());
+        }
+    }
+
+    @Override
+    public void characters(char[] ch, int start, int length)
+            throws SAXException {
+        super.characters(ch, start, length);
+        if (profile.getCount() > nextCheckCount) {
+            checkAndSetLanguage();
+            nextCheckCount = profile.getCount() + CHECK_INTERVAL;
+        }
+    }
+
+    @Override
+    public void endDocument() throws SAXException {
+        super.endDocument();
+        checkAndSetLanguage();
+    }
+
+}

Added: lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingWriter.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingWriter.java?rev=833594&view=auto
==============================================================================
--- lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingWriter.java (added)
+++ lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/language/ProfilingWriter.java Fri Nov  6 22:53:04 2009
@@ -0,0 +1,80 @@
+/*
+ * 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.tika.language;
+
+import java.io.IOException;
+import java.io.Writer;
+
+public class ProfilingWriter extends Writer {
+
+    public static LanguageProfile profile(String content) {
+        ProfilingWriter writer = new ProfilingWriter();
+        char[] ch = content.toCharArray();
+        writer.write(ch, 0, ch.length);
+        return writer.getProfile();
+    }
+
+
+    private final LanguageProfile profile = new LanguageProfile();
+
+    private char[] buffer = new char[] { 0, 0, '_' };
+
+    private int n = 1;
+
+    public LanguageProfile getProfile() {
+        return profile;
+    }
+
+    @Override
+    public void write(char[] cbuf, int off, int len) {
+        for (int i = 0; i < len; i++) {
+            char c = Character.toLowerCase(cbuf[off + i]);
+            if (Character.isLetter(c)) {
+                addLetter(c);
+            } else {
+                addSeparator();
+            }
+        }
+    }
+
+    private void addLetter(char c) {
+        System.arraycopy(buffer, 1, buffer, 0, buffer.length - 1);
+        buffer[buffer.length - 1] = c;
+        n++;
+        if (n >= buffer.length) {
+            profile.add(new String(buffer));
+        }
+    }
+
+    private void addSeparator() {
+        addLetter('_');
+        n = 1;
+    }
+
+    @Override
+    public void close() throws IOException {
+        addSeparator();
+    }
+
+    /**
+     * Ignored.
+     */
+    @Override
+    public void flush() {
+    }
+
+}

Modified: lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/parser/AutoDetectParser.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/parser/AutoDetectParser.java?rev=833594&r1=833593&r2=833594&view=diff
==============================================================================
--- lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/parser/AutoDetectParser.java (original)
+++ lucene/tika/trunk/tika-core/src/main/java/org/apache/tika/parser/AutoDetectParser.java Fri Nov  6 22:53:04 2009
@@ -25,9 +25,11 @@
 import org.apache.tika.config.TikaConfig;
 import org.apache.tika.exception.TikaException;
 import org.apache.tika.io.CountingInputStream;
+import org.apache.tika.language.ProfilingHandler;
 import org.apache.tika.metadata.Metadata;
 import org.apache.tika.mime.MediaType;
 import org.apache.tika.sax.SecureContentHandler;
+import org.apache.tika.sax.TeeContentHandler;
 import org.apache.tika.detect.Detector;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
@@ -98,9 +100,14 @@
         CountingInputStream count = new CountingInputStream(stream);
         SecureContentHandler secure = new SecureContentHandler(handler, count);
 
+        // Automatic language detection
+        ContentHandler profiler = new ProfilingHandler(metadata);
+
         // Parse the document
         try {
-            super.parse(count, secure, metadata, context);
+            super.parse(
+                    count, new TeeContentHandler(secure, profiler),
+                    metadata, context);
         } catch (SAXException e) {
             // Convert zip bomb exceptions to TikaExceptions
             secure.throwIfCauseOf(e);

Copied: lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageIdentifierTest.java (from r829668, lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestLanguageIdentifier.java)
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageIdentifierTest.java?p2=lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageIdentifierTest.java&p1=lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestLanguageIdentifier.java&r1=829668&r2=833594&rev=833594&view=diff
==============================================================================
--- lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/TestLanguageIdentifier.java (original)
+++ lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageIdentifierTest.java Fri Nov  6 22:53:04 2009
@@ -16,17 +16,14 @@
  */
 package org.apache.tika.language;
 
-import java.io.BufferedReader;
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.util.List;
+import java.io.Writer;
 
-import junit.framework.Test;
 import junit.framework.TestCase;
-import junit.framework.TestSuite;
-import junit.textui.TestRunner;
+
+import org.apache.tika.io.IOUtils;
 
 /**
  * JUnit based test of class {@link LanguageIdentifier}.
@@ -34,179 +31,45 @@
  * @author Sami Siren
  * @author Jerome Charron - http://frutch.free.fr/
  */
-public class TestLanguageIdentifier extends TestCase {
-
-    public TestLanguageIdentifier(String testName) {
-        super(testName);
-    }
-
-    public static Test suite() {
-        return new TestSuite(TestLanguageIdentifier.class);
-    }
-
-    public static void main(String[] args) {
-        TestRunner.run(suite());
-    }
-
-    String tokencontent1 = "testaddtoken";
-    String tokencontent2 = "anotherteststring";
-
-    int[] counts1 = { 3, 2, 2, 1, 1, 1, 1, 1 };
-
-    String[] chars1 = { "t", "d", "e", "a", "k", "n", "o", "s" };
-
-    /**
-     * Test analyze method
-     */
-    public void testAnalyze() {
-        String tokencontent = "testmeagain";
-
-        NGramProfile p = new NGramProfile("test", 1, 1);
-        p.analyze(new StringBuilder(tokencontent));
-
-        //test that profile size is ok, eg 9 different NGramEntries "tesmagin"
-        assertEquals(8, p.getSorted().size());
-    }
-
-    /**
-     * Test addNGrams method with StringBuffer argument
-     *
-     */
-    public void testAddNGramsStringBuffer() {
-        String tokencontent = "testmeagain";
-
-        NGramProfile p = new NGramProfile("test", 1, 1);
-        p.add(new StringBuffer(tokencontent));
-
-        //test that profile size is ok, eg 8 different NGramEntries "tesmagin"
-        assertEquals(8, p.getSorted().size());
-
-    }
-
-    /**
-     * test getSorted method
-     */
-    public void testGetSorted() {
-        int[] count = { 4, 3, 1 };
-        String[] ngram = { "a", "b", "c" };
-
-        String teststring = "AAaaBbbC";
-
-        NGramProfile p = new NGramProfile("test", 1, 1);
-        p.analyze(new StringBuilder(teststring));
-
-        //test size of profile
-        assertEquals(3, p.getSorted().size());
-
-        testCounts(p.getSorted(), count);
-        testContents(p.getSorted(), ngram);
-    }
-
-    public void testGetSimilarity() {
-        NGramProfile a = new NGramProfile("a", 1, 1);
-        NGramProfile b = new NGramProfile("b", 1, 1);
+public class LanguageIdentifierTest extends TestCase {
 
-        a.analyze(new StringBuilder(tokencontent1));
-        b.analyze(new StringBuilder(tokencontent2));
-
-        //because of rounding errors might slightly return different results
-        assertEquals(a.getSimilarity(b), b.getSimilarity(a), 0.0000002);
-    }
-
-    public void testExactMatch() {
-        NGramProfile a = new NGramProfile("a", 1, 1);
-
-        a.analyze(new StringBuilder(tokencontent1));
-
-        assertEquals(a.getSimilarity(a), 0, 0);
-    }
-
-
-    public void testIO() {
-        //Create profile and set some contents
-        NGramProfile a = new NGramProfile("a", 1, 1);
-        a.analyze(new StringBuilder(this.tokencontent1));
-
-        NGramProfile b = new NGramProfile("a_from_inputstream", 1, 1);
-
-        //save profile
-        ByteArrayOutputStream os = new ByteArrayOutputStream();
-
-        try {
-            a.save(os);
-            os.close();
-        } catch (Exception e) {
-            fail();
-        }
-
-        //load profile
-        InputStream is = new ByteArrayInputStream(os.toByteArray());
-        try {
-            b.load(is);
-            is.close();
-        } catch (Exception e) {
-            fail();
-        }
-
-        //check it
-        testCounts(b.getSorted(), counts1);
-        testContents(b.getSorted(), chars1);
-    }
-
-    private void testContents(List<NGramEntry> entries, String contents[]) {
-        int c = 0;
-
-        for (NGramEntry nge : entries) {
-            assertEquals(contents[c], nge.getSeq().toString());
-            c++;
+    private static final String[] languages = new String[] {
+        "da", "de", /* "el", */ "en", "es", "fi", "fr", "it", "nl", "pt", "sv"
+    };
+
+    public void testLanguageDetection() throws IOException {
+        for (String language : languages) {
+            ProfilingWriter writer = new ProfilingWriter();
+            writeTo(language, writer);
+            LanguageIdentifier identifier =
+                new LanguageIdentifier(writer.getProfile());
+            assertTrue(identifier.toString(), identifier.isReasonablyCertain());
+            assertEquals(language, identifier.getLanguage());
         }
     }
 
-    private void testCounts(List<NGramEntry> entries, int counts[]) {
-        int c = 0;
-
-        for (NGramEntry nge : entries) {
-            // System.out.println(nge);
-            assertEquals(counts[c], nge.getCount());
-            c++;
+    public void testMixedLanguages() throws IOException {
+        for (String language : languages) {
+            for (String other : languages) {
+                if (!language.equals(other)) {
+                    ProfilingWriter writer = new ProfilingWriter();
+                    writeTo(language, writer);
+                    writeTo(other, writer);
+                    LanguageIdentifier identifier =
+                        new LanguageIdentifier(writer.getProfile());
+                    assertFalse(identifier.isReasonablyCertain());
+                }
+            }
         }
     }
 
-    public void testIdentify() {
+    private void writeTo(String language, Writer writer) throws IOException {
+        InputStream stream =
+            LanguageIdentifierTest.class.getResourceAsStream(language + ".test");
         try {
-            long total = 0;
-            LanguageIdentifier idfr = new LanguageIdentifier();
-            BufferedReader in = new BufferedReader(new InputStreamReader(
-                    this.getClass().getResourceAsStream("test-referencial.txt")));
-            String line = null;
-            while((line = in.readLine()) != null) {
-                String[] tokens = line.split(";");
-                if (!tokens[0].equals("")) {
-                    long start = System.currentTimeMillis();
-                    // Identify the whole file
-                    String lang = idfr.identify(this.getClass().getResourceAsStream(tokens[0]), "UTF-8");
-                    total += System.currentTimeMillis() - start;
-                    assertEquals(tokens[1], lang);
-                    // Then, each line of the file...
-                    BufferedReader testFile = new BufferedReader(
-                            new InputStreamReader(
-                                    this.getClass().getResourceAsStream(tokens[0]), "UTF-8"));
-                    String testLine = null;
-                    while((testLine = testFile.readLine()) != null) {
-                        testLine = testLine.trim();
-                        if (testLine.length() > 256) {
-                            lang = idfr.identify(testLine);
-                            assertEquals(tokens[1], lang);
-                        }
-                    }
-                    testFile.close();
-                }
-            }
-            in.close();
-            System.out.println("Total Time=" + total);
-        } catch(Exception e) {
-            e.printStackTrace();
-            fail(e.toString());
+            IOUtils.copy(new InputStreamReader(stream, "UTF-8"), writer);
+        } finally {
+            stream.close();
         }
     }
 

Added: lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageProfileTest.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageProfileTest.java?rev=833594&view=auto
==============================================================================
--- lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageProfileTest.java (added)
+++ lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/LanguageProfileTest.java Fri Nov  6 22:53:04 2009
@@ -0,0 +1,54 @@
+/*
+ * 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.tika.language;
+
+import java.io.IOException;
+
+import junit.framework.TestCase;
+
+public class LanguageProfileTest extends TestCase {
+
+    public void testLanguageProfile() throws IOException {
+        LanguageProfile foo = new LanguageProfile();
+        assertEquals(0, foo.getCount("foo"));
+
+        foo.add("foo");
+        assertEquals(1, foo.getCount("foo"));
+
+        foo.add("foo", 3);
+        assertEquals(4, foo.getCount("foo"));
+
+        LanguageProfile bar = new LanguageProfile();
+        assertEquals(1.0, foo.distance(bar));
+
+        bar.add("bar");
+        assertEquals(Math.sqrt(2.0), foo.distance(bar));
+
+        bar.add("bar", 3);
+        assertEquals(Math.sqrt(2.0), foo.distance(bar));
+
+        LanguageProfile foobar = new LanguageProfile();
+        assertTrue(foo.distance(foobar) == bar.distance(foobar));
+
+        foobar.add("foo");
+        assertTrue( foo.distance(foobar) < bar.distance(foobar));
+
+        foobar.add("bar");
+        assertTrue(foo.distance(foobar) == bar.distance(foobar));
+    }
+
+}

Modified: lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/ProfilingWriterTest.java
URL: http://svn.apache.org/viewvc/lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/ProfilingWriterTest.java?rev=833594&r1=833593&r2=833594&view=diff
==============================================================================
--- lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/ProfilingWriterTest.java (original)
+++ lucene/tika/trunk/tika-core/src/test/java/org/apache/tika/language/ProfilingWriterTest.java Fri Nov  6 22:53:04 2009
@@ -17,40 +17,25 @@
 package org.apache.tika.language;
 
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-
-import org.apache.tika.io.IOUtils;
 
 import junit.framework.TestCase;
 
 public class ProfilingWriterTest extends TestCase {
 
     public void testProfilingWriter() throws IOException {
-        assertProfile("da");
-        assertProfile("de");
-        assertProfile("el");
-        assertProfile("en");
-        assertProfile("es");
-        assertProfile("fi");
-        assertProfile("fr");
-        assertProfile("it");
-        assertProfile("nl");
-        assertProfile("pt");
-        assertProfile("sv");
-    }
-
-    private void assertProfile(String lang) throws IOException {
-        InputStream stream =
-            ProfilingWriterTest.class.getResourceAsStream(lang + ".test");
-        try {
-            ProfilingWriter writer = new ProfilingWriter();
-            IOUtils.copy(new InputStreamReader(stream, "UTF-8"), writer);
-            NGramProfile profile = writer.getProfile();
-            assertEquals(lang, new LanguageIdentifier(profile).identify());
-        } finally {
-            stream.close();
-        }
+        ProfilingWriter writer = new ProfilingWriter();
+        writer.write(" foo+BAR FooBar\n");
+        writer.close();
+
+        LanguageProfile profile = writer.getProfile();
+        assertEquals(2, profile.getCount("_fo"));
+        assertEquals(2, profile.getCount("foo"));
+        assertEquals(1, profile.getCount("oo_"));
+        assertEquals(1, profile.getCount("oob"));
+        assertEquals(1, profile.getCount("oba"));
+        assertEquals(1, profile.getCount("_ba"));
+        assertEquals(2, profile.getCount("bar"));
+        assertEquals(2, profile.getCount("ar_"));
     }
 
 }