You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by su...@apache.org on 2019/01/15 16:40:14 UTC

[incubator-pinot] branch pinot-text-search updated: Load search index and wire-in the filter operator. Has bare minimum functionality; todo: handle RT, introduce right datatype and search multiple fields

This is an automated email from the ASF dual-hosted git repository.

sunithabeeram pushed a commit to branch pinot-text-search
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/pinot-text-search by this push:
     new b47aa13  Load search index and wire-in the filter operator. Has bare minimum functionality; todo: handle RT, introduce right datatype and search multiple fields
b47aa13 is described below

commit b47aa1399d7774716e3778d664ff670d39ba9b5f
Author: Sunitha Beeram <sb...@linkedin.com>
AuthorDate: Tue Jan 15 08:39:48 2019 -0800

    Load search index and wire-in the filter operator. Has bare minimum functionality; todo: handle RT, introduce right datatype and search multiple fields
---
 .../pinot/common/config/IndexingConfig.java        |   11 +
 .../pinot/common/config/TextSearchIndexConfig.java |   56 ++
 pinot-core/pom.xml                                 |    5 +
 .../java/com/linkedin/pinot/core/common/Block.java |    6 +-
 .../com/linkedin/pinot/core/common/DataSource.java |    8 +-
 .../pinot/core/common/DataSourceMetadata.java      |    2 +
 .../generator/SegmentGeneratorConfig.java          |   10 +-
 .../indexsegment/immutable/ImmutableSegment.java   |    9 +
 .../immutable/ImmutableSegmentImpl.java            |    6 +
 .../immutable/ImmutableSegmentLoader.java          |    2 +-
 .../indexsegment/mutable/MutableSegmentImpl.java   |    2 +-
 .../dociditerators/LuceneDocIdIterator.java        |   93 ++
 .../core/operator/docidsets/LuceneDocIdSet.java    |   77 ++
 .../operator/filter/TextMatchFilterOperator.java   |   26 +-
 .../BaseDictionaryBasedPredicateEvaluator.java     |    3 +
 .../filter/predicate/BasePredicateEvaluator.java   |    3 +
 .../TextMatchPredicateEvaluatorFactory.java        |   10 +-
 .../NoDictionaryBasedInvertedIndexCreator.java     |    5 +
 .../creator/impl/SegmentColumnarIndexCreator.java  |   16 +-
 .../impl/textsearch/LuceneIndexCreator.java        |   20 +-
 .../impl/textsearch/TextSearchIndexConfig.java     |   54 --
 .../textsearch/TextSearchIndexCreatorFactory.java  |    9 +-
 .../segment/index/column/ColumnIndexContainer.java |    8 +-
 .../index/column/PhysicalColumnIndexContainer.java |   31 +-
 .../index/data/source/ColumnDataSource.java        |   22 +-
 .../segment/index/loader/IndexLoadingConfig.java   |    8 +
 .../loader/invertedindex/InvertedIndexHandler.java |   20 +-
 .../index/readers/LuceneSearchIndexReader.java     |   63 ++
 .../readers/SearchIndexReader.java}                |   14 +-
 .../virtualcolumn/VirtualColumnIndexContainer.java |    6 +
 .../v2/store/StarTreeDimensionDataSource.java      |   11 +
 .../v2/store/StarTreeMetricDataSource.java         |   11 +
 .../core/common/RealtimeNoDictionaryTest.java      |    8 +-
 .../pinot/queries/TextMatchQueriesTest.java        |  188 ++++
 pinot-core/src/test/resources/data/logfile         | 1000 ++++++++++++++++++++
 35 files changed, 1713 insertions(+), 110 deletions(-)

diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
index c3a45900..1e0eb81 100644
--- a/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/IndexingConfig.java
@@ -78,6 +78,9 @@ public class IndexingConfig {
   @ConfigKey("starTreeIndexConfigs")
   private List<StarTreeIndexConfig> _starTreeIndexConfigs;
 
+  @ConfigKey("textSearchIndexConfig")
+  private TextSearchIndexConfig _textSearchIndexConfig;
+
   @ConfigKey("segmentPartitionConfig")
   private SegmentPartitionConfig _segmentPartitionConfig;
 
@@ -205,6 +208,14 @@ public class IndexingConfig {
     _starTreeIndexConfigs = starTreeIndexConfigs;
   }
 
+  public TextSearchIndexConfig getTextSearchIndexConfig() {
+    return _textSearchIndexConfig;
+  }
+
+  public void setTextSearchIndexConfig(TextSearchIndexConfig config) {
+    _textSearchIndexConfig = config;
+  }
+
   public void setSegmentPartitionConfig(SegmentPartitionConfig config) {
     _segmentPartitionConfig = config;
   }
diff --git a/pinot-common/src/main/java/com/linkedin/pinot/common/config/TextSearchIndexConfig.java b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TextSearchIndexConfig.java
new file mode 100644
index 0000000..ec8e703
--- /dev/null
+++ b/pinot-common/src/main/java/com/linkedin/pinot/common/config/TextSearchIndexConfig.java
@@ -0,0 +1,56 @@
+/**
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * 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 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 com.linkedin.pinot.common.config;
+import java.io.File;
+import org.codehaus.jackson.annotate.JsonIgnoreProperties;
+
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TextSearchIndexConfig {
+
+  @ConfigKey("type")
+  @ConfigDoc("Type of indexer to use")
+  private String type;
+
+  @ConfigKey("store")
+  @ConfigDoc("Whether column should be stored. If false, it will just be indexed")
+  private boolean store;
+
+  public TextSearchIndexConfig(String type, boolean store) {
+    this.type = type;
+    this.store = store;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public boolean shouldStore() {
+    return store;
+  }
+
+  public void setStore(boolean store) {
+    this.store = store;
+  }
+
+  public static TextSearchIndexConfig getDefaultConfig() {
+    return new TextSearchIndexConfig("LUCENE", true);
+  }
+}
diff --git a/pinot-core/pom.xml b/pinot-core/pom.xml
index de451b5..f526453 100644
--- a/pinot-core/pom.xml
+++ b/pinot-core/pom.xml
@@ -151,6 +151,11 @@
       <version>7.6.0</version>
     </dependency>
     <dependency>
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-queryparser</artifactId>
+      <version>7.6.0</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
     </dependency>
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/common/Block.java b/pinot-core/src/main/java/com/linkedin/pinot/core/common/Block.java
index a050918..dd1af24 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/common/Block.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/common/Block.java
@@ -17,9 +17,9 @@ package com.linkedin.pinot.core.common;
 
 /**
  *
- * A block represents a set of rows.A segment will contain one or more blocks
- * Currently, it assumes only one column per block. We might change this in
- * future
+ * A block represents a set of rows. A segment will contain one or more blocks.
+ * Currently, it assumes only one column per block. We might change this in the
+ * future.
  */
 public interface Block {
 
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSource.java b/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSource.java
index 6397024..7c03de5 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSource.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSource.java
@@ -19,6 +19,8 @@ import com.linkedin.pinot.core.operator.BaseOperator;
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
+
 
 public abstract class DataSource extends BaseOperator {
 
@@ -26,7 +28,9 @@ public abstract class DataSource extends BaseOperator {
 
   public abstract InvertedIndexReader getInvertedIndex();
 
+  public abstract SearchIndexReader getSearchIndex();
+
   public abstract Dictionary getDictionary();
-  
-  public abstract BloomFilterReader getBloomFilter();  
+
+  public abstract BloomFilterReader getBloomFilter();
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSourceMetadata.java b/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSourceMetadata.java
index d7796ce..75c4fc1 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSourceMetadata.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/common/DataSourceMetadata.java
@@ -32,6 +32,8 @@ public interface DataSourceMetadata {
 
   boolean hasInvertedIndex();
 
+  boolean hasSearchIndex();
+
   boolean hasDictionary();
 
   int getCardinality();
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
index 73e144b..9a527a4 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/generator/SegmentGeneratorConfig.java
@@ -30,7 +30,7 @@ import com.linkedin.pinot.core.data.readers.CSVRecordReaderConfig;
 import com.linkedin.pinot.core.data.readers.FileFormat;
 import com.linkedin.pinot.core.data.readers.RecordReaderConfig;
 import com.linkedin.pinot.core.io.compression.ChunkCompressorFactory;
-import com.linkedin.pinot.core.segment.creator.impl.textsearch.TextSearchIndexConfig;
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.core.segment.name.DefaultSegmentNameGenerator;
 import com.linkedin.pinot.core.segment.name.SegmentNameGenerator;
 import com.linkedin.pinot.core.startree.v2.builder.StarTreeV2BuilderConfig;
@@ -106,7 +106,7 @@ public class SegmentGeneratorConfig {
   private String _simpleDateFormat = null;
   // Use on-heap or off-heap memory to generate index (currently only affect inverted index and star-tree v2)
   private boolean _onHeap = false;
-  private Map<String, TextSearchIndexConfig> _textSearchIndexConfigs = new HashMap<>();
+  private TextSearchIndexConfig _textSearchIndexConfig;
 
   public SegmentGeneratorConfig() {
   }
@@ -573,11 +573,11 @@ public class SegmentGeneratorConfig {
   }
 
   public void setTextSearchIndexConfig(TextSearchIndexConfig config) {
-    _textSearchIndexConfigs.put(config.getColumnName(),config);
+    _textSearchIndexConfig = config;
   }
 
-  public Map<String, TextSearchIndexConfig> getTextSearchIndexConfigs() {
-    return _textSearchIndexConfigs;
+  public TextSearchIndexConfig getTextSearchIndexConfig() {
+    return _textSearchIndexConfig;
   }
 
   @JsonIgnore
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegment.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegment.java
index d86a2b5..8b22161 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegment.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegment.java
@@ -20,6 +20,7 @@ import com.linkedin.pinot.core.io.reader.DataFileReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.ImmutableDictionaryReader;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 
 
 public interface ImmutableSegment extends IndexSegment {
@@ -49,6 +50,14 @@ public interface ImmutableSegment extends IndexSegment {
   InvertedIndexReader getInvertedIndex(String column);
 
   /**
+   * Returns the search index for the given column.
+   *
+   * @param column Column name
+   * @return Search index for the given column, or null if the given column does not have one
+   */
+  SearchIndexReader getSearchIndex(String column);
+
+  /**
    * Returns the total size of the segment in bytes.
    *
    * @return Size of the segment in bytes
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
index 3edcc3a..1f40d63 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentImpl.java
@@ -24,6 +24,7 @@ import com.linkedin.pinot.core.segment.index.column.ColumnIndexContainer;
 import com.linkedin.pinot.core.segment.index.data.source.ColumnDataSource;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 import com.linkedin.pinot.core.segment.store.SegmentDirectory;
 import com.linkedin.pinot.core.startree.v2.StarTreeV2;
 import com.linkedin.pinot.core.startree.v2.store.StarTreeIndexContainer;
@@ -70,6 +71,11 @@ public class ImmutableSegmentImpl implements ImmutableSegment {
   }
 
   @Override
+  public SearchIndexReader getSearchIndex(String column) {
+    return _indexContainerMap.get(column).getSearchIndex();
+  }
+
+  @Override
   public long getSegmentSizeBytes() {
     return _segmentDirectory.getDiskSizeBytes();
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
index b1313bf..40aa0fa 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/immutable/ImmutableSegmentLoader.java
@@ -110,7 +110,7 @@ public class ImmutableSegmentLoader {
     Map<String, ColumnIndexContainer> indexContainerMap = new HashMap<>();
     for (Map.Entry<String, ColumnMetadata> entry : segmentMetadata.getColumnMetadataMap().entrySet()) {
       indexContainerMap.put(entry.getKey(),
-          new PhysicalColumnIndexContainer(segmentReader, entry.getValue(), indexLoadingConfig));
+          new PhysicalColumnIndexContainer(segmentReader, indexDir, entry.getValue(), indexLoadingConfig));
     }
 
     if (schema == null) {
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/mutable/MutableSegmentImpl.java b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
index 9bd9de5..3ad28f4 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/indexsegment/mutable/MutableSegmentImpl.java
@@ -389,7 +389,7 @@ public class MutableSegmentImpl implements MutableSegment {
     if (!_schema.isVirtualColumn(columnName)) {
       return new ColumnDataSource(_schema.getFieldSpecFor(columnName), _numDocsIndexed, _maxNumValuesMap.get(columnName),
           _indexReaderWriterMap.get(columnName), _invertedIndexMap.get(columnName), _dictionaryMap.get(columnName),
-          _bloomFilterMap.get(columnName));
+          _bloomFilterMap.get(columnName), null);
     } else {
       return getVirtualDataSource(columnName);
     }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/dociditerators/LuceneDocIdIterator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/dociditerators/LuceneDocIdIterator.java
new file mode 100644
index 0000000..fca3978
--- /dev/null
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/dociditerators/LuceneDocIdIterator.java
@@ -0,0 +1,93 @@
+/**
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * 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 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 com.linkedin.pinot.core.operator.dociditerators;
+
+import com.linkedin.pinot.core.common.Constants;
+import org.apache.lucene.search.ScoreDoc;
+import org.roaringbitmap.IntIterator;
+import sun.util.resources.cldr.so.CurrencyNames_so;
+
+
+public final class LuceneDocIdIterator implements IndexBasedDocIdIterator {
+  private int _endDocId = Integer.MAX_VALUE;
+  private int _startDocId;
+  private int _currentDocId = -1;
+  private int _currentIndex = -1;
+  private final ScoreDoc[] _scoreDocs;
+
+  public LuceneDocIdIterator(ScoreDoc[] docs) {
+    _scoreDocs = docs;
+  }
+
+  @Override
+  public int currentDocId() {
+    return _currentDocId;
+  }
+
+  public void setStartDocId(int startDocId) {
+    this._startDocId = startDocId;
+  }
+
+  public void setEndDocId(int endDocId) {
+    this._endDocId = endDocId;
+  }
+
+  @Override
+  public int next() {
+    // Empty?
+    if (_currentDocId == Constants.EOF || ++_currentIndex == _scoreDocs.length) {
+      _currentDocId = Constants.EOF;
+      return Constants.EOF;
+    }
+
+    _currentDocId = _scoreDocs[_currentIndex].doc;
+
+    // Advance to startDocId if necessary
+    while(_currentDocId < _startDocId && ++_currentIndex < _scoreDocs.length) {
+      _currentDocId = _scoreDocs[_currentIndex].doc;
+    }
+
+    // Current docId outside of the valid range?
+    if (_currentDocId < _startDocId || _endDocId < _currentDocId) {
+      _currentDocId = Constants.EOF;
+    }
+
+    return _currentDocId;
+  }
+
+  @Override
+  public int advance(int targetDocId) {
+    if (targetDocId < _currentDocId) {
+      throw new IllegalArgumentException("Trying to move backwards to docId " + targetDocId +
+          ", current position " + _currentDocId);
+    }
+
+    if (_currentDocId == targetDocId) {
+      return _currentDocId;
+    } else {
+      int curr = next();
+      while(curr < targetDocId && curr != Constants.EOF) {
+        curr = next();
+      }
+      return curr;
+    }
+  }
+
+  public String toString() {
+    return LuceneDocIdIterator.class.getSimpleName();
+  }
+
+}
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/docidsets/LuceneDocIdSet.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/docidsets/LuceneDocIdSet.java
new file mode 100644
index 0000000..3ab0c42
--- /dev/null
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/docidsets/LuceneDocIdSet.java
@@ -0,0 +1,77 @@
+/**
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * 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 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 com.linkedin.pinot.core.operator.docidsets;
+
+import com.linkedin.pinot.core.common.BlockDocIdIterator;
+import com.linkedin.pinot.core.operator.dociditerators.BitmapDocIdIterator;
+import com.linkedin.pinot.core.operator.dociditerators.LuceneDocIdIterator;
+import com.linkedin.pinot.core.segment.creator.impl.textsearch.LuceneIndexCreator;
+import org.apache.lucene.search.ScoreDoc;
+import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
+
+public class LuceneDocIdSet implements FilterBlockDocIdSet {
+  private final ScoreDoc[] _docs;
+  private int _startDocId;
+  // Inclusive
+  private int _endDocId;
+
+  public LuceneDocIdSet(ScoreDoc[] docs, int startDocId, int endDocId) {
+    _docs = docs;
+    _startDocId = startDocId;
+    _endDocId = endDocId;
+  }
+
+  @Override
+  public int getMinDocId() {
+    return _startDocId;
+  }
+
+  @Override
+  public int getMaxDocId() {
+    return _endDocId;
+  }
+
+  @Override
+  public void setStartDocId(int startDocId) {
+    _startDocId = startDocId;
+  }
+
+  @Override
+  public void setEndDocId(int endDocId) {
+    _endDocId = endDocId;
+  }
+
+  @Override
+  public long getNumEntriesScannedInFilter() {
+    return 0L;
+  }
+
+  @Override
+  public BlockDocIdIterator iterator() {
+    LuceneDocIdIterator docIdIterator = new LuceneDocIdIterator(_docs);
+    docIdIterator.setStartDocId(_startDocId);
+    docIdIterator.setEndDocId(_endDocId);
+    return docIdIterator;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public <T> T getRaw() {
+    return null;
+  }
+}
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/TextMatchFilterOperator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/TextMatchFilterOperator.java
index 930c669..ed81640 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/TextMatchFilterOperator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/TextMatchFilterOperator.java
@@ -18,9 +18,13 @@ package com.linkedin.pinot.core.operator.filter;
 import com.google.common.base.Preconditions;
 import com.linkedin.pinot.core.common.DataSource;
 import com.linkedin.pinot.core.operator.blocks.FilterBlock;
+import com.linkedin.pinot.core.operator.docidsets.ArrayBasedDocIdSet;
 import com.linkedin.pinot.core.operator.docidsets.BitmapDocIdSet;
+import com.linkedin.pinot.core.operator.docidsets.LuceneDocIdSet;
 import com.linkedin.pinot.core.operator.filter.predicate.PredicateEvaluator;
-import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.operator.filter.predicate.TextMatchPredicateEvaluatorFactory.*;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
+import org.apache.lucene.search.TopDocs;
 import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,14 +37,14 @@ public class TextMatchFilterOperator extends BaseFilterOperator {
   private static final Logger LOGGER = LoggerFactory.getLogger(TextMatchFilterOperator.class);
   private static final String OPERATOR_NAME = "TextMatchFilterOperator";
 
-  private final PredicateEvaluator _predicateEvaluator;
+  private final String _query;
+  private final String _options;
   private final DataSource _dataSource;
-  private final ImmutableRoaringBitmap[] _bitmaps;
   private final int _startDocId;
   // TODO: change it to exclusive
   // Inclusive
   private final int _endDocId;
-  private final boolean _exclusive;
+  private TopDocs _docs;
 
   TextMatchFilterOperator(PredicateEvaluator predicateEvaluator, DataSource dataSource, int startDocId,
                           int endDocId) {
@@ -50,18 +54,24 @@ public class TextMatchFilterOperator extends BaseFilterOperator {
     // If predicate is always evaluated as true, use MatchAllFilterOperator; if predicate is always evaluated as false,
     // use EmptyFilterOperator.
     Preconditions.checkArgument(!predicateEvaluator.isAlwaysTrue() && !predicateEvaluator.isAlwaysFalse());
+    Preconditions.checkArgument(predicateEvaluator instanceof RawValueBasedTextMatchPredicateEvaluator);
 
-    _predicateEvaluator = predicateEvaluator;
+    RawValueBasedTextMatchPredicateEvaluator evaluator = (RawValueBasedTextMatchPredicateEvaluator) predicateEvaluator;
+    _query = evaluator.getQueryString();
+    _options = evaluator.getQueryOptions();
     _dataSource = dataSource;
-    _bitmaps = null;
     _startDocId = startDocId;
     _endDocId = endDocId;
-    _exclusive = predicateEvaluator.isExclusive();
   }
 
   @Override
   protected FilterBlock getNextBlock() {
-    throw new UnsupportedOperationException("WIP");
+
+    if (_docs == null) {
+      SearchIndexReader<TopDocs> searchIndex = _dataSource.getSearchIndex();
+      _docs = searchIndex.getDocIds(_query, _options);
+    }
+    return new FilterBlock(new LuceneDocIdSet(_docs.scoreDocs, _startDocId, _endDocId));
   }
 
   @Override
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BaseDictionaryBasedPredicateEvaluator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BaseDictionaryBasedPredicateEvaluator.java
index 0d3de77..1b232e1 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BaseDictionaryBasedPredicateEvaluator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BaseDictionaryBasedPredicateEvaluator.java
@@ -15,6 +15,9 @@
  */
 package com.linkedin.pinot.core.operator.filter.predicate;
 
+import com.linkedin.pinot.core.common.Predicate;
+
+
 public abstract class BaseDictionaryBasedPredicateEvaluator extends BasePredicateEvaluator {
   protected boolean _alwaysTrue;
   protected boolean _alwaysFalse;
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BasePredicateEvaluator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BasePredicateEvaluator.java
index 2a4d296..76ac9da 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BasePredicateEvaluator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/BasePredicateEvaluator.java
@@ -15,6 +15,9 @@
  */
 package com.linkedin.pinot.core.operator.filter.predicate;
 
+import com.linkedin.pinot.core.common.Predicate;
+
+
 public abstract class BasePredicateEvaluator implements PredicateEvaluator {
 
   @Override
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/TextMatchPredicateEvaluatorFactory.java b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/TextMatchPredicateEvaluatorFactory.java
index 556ffbc..2d1c9ba 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/TextMatchPredicateEvaluatorFactory.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/operator/filter/predicate/TextMatchPredicateEvaluatorFactory.java
@@ -39,7 +39,7 @@ public class TextMatchPredicateEvaluatorFactory {
     return new RawValueBasedTextMatchPredicateEvaluator(textMatchPredicate);
   }
 
-  private static final class RawValueBasedTextMatchPredicateEvaluator extends BaseRawValueBasedPredicateEvaluator {
+  public static final class RawValueBasedTextMatchPredicateEvaluator extends BaseRawValueBasedPredicateEvaluator {
     String _query;
     String _options;
     public RawValueBasedTextMatchPredicateEvaluator(TextMatchPredicate textMatchPredicate) {
@@ -56,5 +56,13 @@ public class TextMatchPredicateEvaluatorFactory {
     public boolean applySV(String value) {
       throw new UnsupportedOperationException("Text Match is not supported via scanning, its supported only via inverted index");
     }
+
+    public String getQueryString() {
+      return _query;
+    }
+
+    public String getQueryOptions() {
+      return _options;
+    }
   }
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java
index a91c7d8..2a5944b 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java
@@ -24,4 +24,9 @@ public interface NoDictionaryBasedInvertedIndexCreator extends InvertedIndexCrea
    * Add a document field to the index
    */
   void add(Object doc);
+
+  /**
+   * Check if index exists for column.
+   */
+  boolean indexExists(String column);
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
index bd43647..f7033b0 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/SegmentColumnarIndexCreator.java
@@ -44,7 +44,7 @@ import com.linkedin.pinot.core.segment.creator.impl.fwd.SingleValueUnsortedForwa
 import com.linkedin.pinot.core.segment.creator.impl.fwd.SingleValueVarByteRawIndexCreator;
 import com.linkedin.pinot.core.segment.creator.impl.inv.OffHeapBitmapInvertedIndexCreator;
 import com.linkedin.pinot.core.segment.creator.impl.inv.OnHeapBitmapInvertedIndexCreator;
-import com.linkedin.pinot.core.segment.creator.impl.textsearch.TextSearchIndexConfig;
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.core.segment.creator.impl.textsearch.TextSearchIndexCreatorFactory;
 import com.linkedin.pinot.startree.hll.HllConfig;
 import java.io.File;
@@ -190,12 +190,12 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
 
         boolean createFwdIndex = true;
         if(createInvertedIndex && type == FieldSpec.DataType.TEXT) {
-          TextSearchIndexConfig config = segmentCreationSpec.getTextSearchIndexConfigs().get(columnName);
+          TextSearchIndexConfig config = segmentCreationSpec.getTextSearchIndexConfig();
           if (config == null) {
-            config = TextSearchIndexConfig.getDefaultConfig(columnName, _indexDir);
+            config = TextSearchIndexConfig.getDefaultConfig();
           }
           createFwdIndex = config.shouldStore();
-          _noDictionaryIndexCreatorMap.put(columnName, TextSearchIndexCreatorFactory.createSearchIndexer(config));
+          _noDictionaryIndexCreatorMap.put(columnName, TextSearchIndexCreatorFactory.createSearchIndexer(config, columnName, _indexDir));
         }
 
         if (createFwdIndex) {
@@ -318,8 +318,8 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
     for (InvertedIndexCreator invertedIndexCreator : _invertedIndexCreatorMap.values()) {
       invertedIndexCreator.seal();
     }
-    for (NoDictionaryBasedInvertedIndexCreator docIndexCreator : _noDictionaryIndexCreatorMap.values()) {
-      docIndexCreator.seal();
+    for (NoDictionaryBasedInvertedIndexCreator noDictIndexCreator : _noDictionaryIndexCreatorMap.values()) {
+      noDictIndexCreator.seal();
     }
     writeMetadata();
   }
@@ -569,8 +569,8 @@ public class SegmentColumnarIndexCreator implements SegmentCreator {
     for (InvertedIndexCreator invertedIndexCreator : _invertedIndexCreatorMap.values()) {
       invertedIndexCreator.close();
     }
-    for (NoDictionaryBasedInvertedIndexCreator docIndexCreator : _noDictionaryIndexCreatorMap.values()) {
-      docIndexCreator.close();
+    for (NoDictionaryBasedInvertedIndexCreator noDictIndexCreator : _noDictionaryIndexCreatorMap.values()) {
+      noDictIndexCreator.close();
     }
   }
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/LuceneIndexCreator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/LuceneIndexCreator.java
index f630e85..b58e7a9 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/LuceneIndexCreator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/LuceneIndexCreator.java
@@ -15,6 +15,7 @@
  */
 package com.linkedin.pinot.core.segment.creator.impl.textsearch;
 
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.core.segment.creator.NoDictionaryBasedInvertedIndexCreator;
 import java.io.File;
 import java.io.IOException;
@@ -30,28 +31,33 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 
-class LuceneIndexCreator implements NoDictionaryBasedInvertedIndexCreator {
+public class LuceneIndexCreator implements NoDictionaryBasedInvertedIndexCreator {
+
+  public static final String DEFAULT_FIELD = "TEXT";
 
   private static final Logger LOGGER = LoggerFactory.getLogger(LuceneIndexCreator.class);
 
   public static final String VERSION = "7.6.0";
   // Index file will be flushed after reaching this threshold
   private static final int MAX_BUFFER_SIZE_MB = 500;
-  private static final String DEFAULT_FIELD = "TEXT";
   private static final Field.Store DEFAULT_STORE = Field.Store.NO;
 
   private final TextSearchIndexConfig _config;
+  private final String _columnName;
+  private final File _indexDir;
   private final StandardAnalyzer _analyzer;
   private final IndexWriter _writer;
   private final IndexWriterConfig _indexWriterConfig;
   private final Directory _indexDirectory;
 
-  public LuceneIndexCreator(TextSearchIndexConfig config) {
+  public LuceneIndexCreator(TextSearchIndexConfig config, String columnName, File indexDir) {
     _config = config;
+    _columnName = columnName;
+    _indexDir = indexDir;
     _analyzer = new StandardAnalyzer();
     _indexWriterConfig = new IndexWriterConfig(_analyzer);
     _indexWriterConfig.setRAMBufferSizeMB(MAX_BUFFER_SIZE_MB);
-    File dir = new File(config.getIndexDir().getPath() + "/" + _config.getColumnName());
+    File dir = new File(_indexDir.getPath() + "/" + _columnName);
     try {
       _indexDirectory = FSDirectory.open(dir.toPath());
       _writer = new IndexWriter(_indexDirectory, _indexWriterConfig);
@@ -79,6 +85,12 @@ class LuceneIndexCreator implements NoDictionaryBasedInvertedIndexCreator {
   }
 
   @Override
+  public boolean indexExists(String column) {
+    // todo: add appropriate functionality
+    return true;
+  }
+
+  @Override
   public void close() throws IOException {
     _writer.close();
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexConfig.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexConfig.java
deleted file mode 100644
index d8a4fbd..0000000
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexConfig.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
- *
- * 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 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 com.linkedin.pinot.core.segment.creator.impl.textsearch;
-
-import java.io.File;
-
-
-public class TextSearchIndexConfig {
-
-  private final TextSearchIndexType _type;
-  private final boolean _store;
-  private final File _indexDir;
-  private final String _columnName;
-
-  public TextSearchIndexConfig(TextSearchIndexType type, boolean store, File indexDir, String columnName) {
-    _type = type;
-    _store = store;
-    _indexDir = indexDir;
-    _columnName = columnName;
-  }
-
-  public TextSearchIndexType getType() {
-    return _type;
-  }
-
-  public boolean shouldStore() {
-    return _store;
-  }
-
-  public File getIndexDir() {
-    return _indexDir;
-  }
-
-  public String getColumnName() {
-    return _columnName;
-  }
-
-  public static TextSearchIndexConfig getDefaultConfig(String columnName, File indexDir) {
-    return new TextSearchIndexConfig(TextSearchIndexType.LUCENE, true, indexDir, columnName);
-  }
-}
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexCreatorFactory.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexCreatorFactory.java
index d31a183..9ad90dc 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexCreatorFactory.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/impl/textsearch/TextSearchIndexCreatorFactory.java
@@ -15,16 +15,19 @@
  */
 package com.linkedin.pinot.core.segment.creator.impl.textsearch;
 
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.core.segment.creator.NoDictionaryBasedInvertedIndexCreator;
+import java.io.File;
 
 
 public class TextSearchIndexCreatorFactory {
 
-   public static NoDictionaryBasedInvertedIndexCreator createSearchIndexer(TextSearchIndexConfig config) {
+   public static NoDictionaryBasedInvertedIndexCreator createSearchIndexer(TextSearchIndexConfig config,
+       String columnName, File indexDir) {
 
      switch (config.getType()) {
-       case LUCENE:
-         return new LuceneIndexCreator(config);
+       case "LUCENE":
+         return new LuceneIndexCreator(config, columnName, indexDir);
        default:
          throw new IllegalArgumentException("Unsupported TextSearchIndexType " + config.getType());
      }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/ColumnIndexContainer.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/ColumnIndexContainer.java
index 6e4e515..4dd9f54 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/ColumnIndexContainer.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/ColumnIndexContainer.java
@@ -19,6 +19,7 @@ import com.linkedin.pinot.core.io.reader.DataFileReader;
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 
 
 /**
@@ -37,9 +38,14 @@ public interface ColumnIndexContainer {
   InvertedIndexReader getInvertedIndex();
 
   /**
+   * Returns the search index for the column or {@code null} if it does not exist.
+   */
+  SearchIndexReader getSearchIndex();
+
+  /**
    * Returns the dictionary for the column, or {@code null} if it does not exist.
    */
   Dictionary getDictionary();
-  
+
   BloomFilterReader getBloomFilter();
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/PhysicalColumnIndexContainer.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/PhysicalColumnIndexContainer.java
index da1e9cd..b3b8f38 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/PhysicalColumnIndexContainer.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/column/PhysicalColumnIndexContainer.java
@@ -15,6 +15,7 @@
  */
 package com.linkedin.pinot.core.segment.index.column;
 
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.core.io.reader.DataFileReader;
 import com.linkedin.pinot.core.io.reader.SingleColumnSingleValueReader;
@@ -35,15 +36,18 @@ import com.linkedin.pinot.core.segment.index.readers.ImmutableDictionaryReader;
 import com.linkedin.pinot.core.segment.index.readers.IntDictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
 import com.linkedin.pinot.core.segment.index.readers.LongDictionary;
+import com.linkedin.pinot.core.segment.index.readers.LuceneSearchIndexReader;
 import com.linkedin.pinot.core.segment.index.readers.OnHeapDoubleDictionary;
 import com.linkedin.pinot.core.segment.index.readers.OnHeapFloatDictionary;
 import com.linkedin.pinot.core.segment.index.readers.OnHeapIntDictionary;
 import com.linkedin.pinot.core.segment.index.readers.OnHeapLongDictionary;
 import com.linkedin.pinot.core.segment.index.readers.OnHeapStringDictionary;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 import com.linkedin.pinot.core.segment.index.readers.StringDictionary;
 import com.linkedin.pinot.core.segment.memory.PinotDataBuffer;
 import com.linkedin.pinot.core.segment.store.ColumnIndexType;
 import com.linkedin.pinot.core.segment.store.SegmentDirectory;
+import java.io.File;
 import java.io.IOException;
 
 import org.slf4j.Logger;
@@ -55,11 +59,12 @@ public final class PhysicalColumnIndexContainer implements ColumnIndexContainer
 
   private final DataFileReader _forwardIndex;
   private final InvertedIndexReader _invertedIndex;
+  private final SearchIndexReader _searchIndex;
   private final ImmutableDictionaryReader _dictionary;
   private final BloomFilterReader _bloomFilterReader;
 
-  public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, ColumnMetadata metadata,
-      IndexLoadingConfig indexLoadingConfig) throws IOException {
+  public PhysicalColumnIndexContainer(SegmentDirectory.Reader segmentReader, File indexDir,
+      ColumnMetadata metadata, IndexLoadingConfig indexLoadingConfig) throws IOException {
     String columnName = metadata.getColumnName();
     boolean loadInvertedIndex = false;
     boolean loadOnHeapDictionary = false;
@@ -73,6 +78,7 @@ public final class PhysicalColumnIndexContainer implements ColumnIndexContainer
 
     FieldSpec.DataType type = metadata.getDataType();
     if (metadata.hasDictionary()) {
+      _searchIndex = null;
       //bloom filter
       if (loadBloomFilter) {
         PinotDataBuffer bloomFilterBuffer = segmentReader.getIndexFor(columnName, ColumnIndexType.BLOOM_FILTER);
@@ -117,8 +123,22 @@ public final class PhysicalColumnIndexContainer implements ColumnIndexContainer
       _invertedIndex = null;
       if (loadInvertedIndex) {
         if (type == FieldSpec.DataType.TEXT) {
-          // TODO: load text-search inverted index
+          TextSearchIndexConfig searchConfig = indexLoadingConfig.getTextSearchIndexConfig();
+          if (searchConfig == null) {
+            searchConfig = TextSearchIndexConfig.getDefaultConfig();
+          }
+          switch (searchConfig.getType()) {
+            case "LUCENE":
+              _searchIndex = new LuceneSearchIndexReader(columnName, indexDir);
+              break;
+            default:
+              throw new RuntimeException("Unsupported Text search index type " + searchConfig.getType());
+          }
+        } else {
+          _searchIndex = null;
         }
+      } else {
+        _searchIndex = null;
       }
     }
   }
@@ -134,6 +154,11 @@ public final class PhysicalColumnIndexContainer implements ColumnIndexContainer
   }
 
   @Override
+  public SearchIndexReader getSearchIndex() {
+    return _searchIndex;
+  }
+
+  @Override
   public ImmutableDictionaryReader getDictionary() {
     return _dictionary;
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/data/source/ColumnDataSource.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/data/source/ColumnDataSource.java
index 6f89e71..fa33056 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/data/source/ColumnDataSource.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/data/source/ColumnDataSource.java
@@ -33,6 +33,7 @@ import com.linkedin.pinot.core.segment.index.column.ColumnIndexContainer;
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 
 
 public final class ColumnDataSource extends DataSource {
@@ -44,6 +45,7 @@ public final class ColumnDataSource extends DataSource {
   private final int _maxNumMultiValues;
   private final DataFileReader _forwardIndex;
   private final InvertedIndexReader _invertedIndex;
+  private final SearchIndexReader _searchIndex;
   private final Dictionary _dictionary;
   private final BloomFilterReader _bloomFilter;
   private final int _cardinality;
@@ -56,21 +58,22 @@ public final class ColumnDataSource extends DataSource {
     this(metadata.getColumnName(), metadata.getDataType(), metadata.isSingleValue(), metadata.isSorted(),
         metadata.getTotalDocs(), metadata.getMaxNumberOfMultiValues(), indexContainer.getForwardIndex(),
         indexContainer.getInvertedIndex(), indexContainer.getDictionary(), indexContainer.getBloomFilter(),
-        metadata.getCardinality());
+        indexContainer.getSearchIndex(), metadata.getCardinality());
   }
 
   /**
    * For REALTIME segment.
    */
   public ColumnDataSource(FieldSpec fieldSpec, int numDocs, int maxNumMultiValues, DataFileReader forwardIndex,
-      InvertedIndexReader invertedIndex, MutableDictionary dictionary, BloomFilterReader bloomFilter) {
+      InvertedIndexReader invertedIndex, MutableDictionary dictionary, BloomFilterReader bloomFilter,
+      SearchIndexReader searchIndex) {
     this(fieldSpec.getName(), fieldSpec.getDataType(), fieldSpec.isSingleValueField(), false, numDocs,
-        maxNumMultiValues, forwardIndex, invertedIndex, dictionary, bloomFilter, Constants.UNKNOWN_CARDINALITY);
+        maxNumMultiValues, forwardIndex, invertedIndex, dictionary, bloomFilter, searchIndex, Constants.UNKNOWN_CARDINALITY);
   }
 
   private ColumnDataSource(String columnName, FieldSpec.DataType dataType, boolean isSingleValue, boolean isSorted,
       int numDocs, int maxNumMultiValues, DataFileReader forwardIndex, InvertedIndexReader invertedIndex,
-      Dictionary dictionary, BloomFilterReader bloomFilterReader, int cardinality) {
+      Dictionary dictionary, BloomFilterReader bloomFilterReader, SearchIndexReader searchIndex, int cardinality) {
     // Sanity check
     if (isSingleValue) {
       Preconditions.checkState(forwardIndex instanceof SingleColumnSingleValueReader);
@@ -95,6 +98,7 @@ public final class ColumnDataSource extends DataSource {
     _maxNumMultiValues = maxNumMultiValues;
     _forwardIndex = forwardIndex;
     _invertedIndex = invertedIndex;
+    _searchIndex = searchIndex;
     _dictionary = dictionary;
     _bloomFilter = bloomFilterReader;
     _cardinality = cardinality;
@@ -131,6 +135,11 @@ public final class ColumnDataSource extends DataSource {
       }
 
       @Override
+      public boolean hasSearchIndex() {
+        return _searchIndex != null;
+      }
+
+      @Override
       public boolean hasDictionary() {
         return _dictionary != null;
       }
@@ -153,6 +162,11 @@ public final class ColumnDataSource extends DataSource {
   }
 
   @Override
+  public SearchIndexReader getSearchIndex() {
+    return _searchIndex;
+  }
+
+  @Override
   public Dictionary getDictionary() {
     return _dictionary;
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
index 7c54c90..49e567a 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/IndexLoadingConfig.java
@@ -18,6 +18,7 @@ package com.linkedin.pinot.core.segment.index.loader;
 import com.google.common.annotations.VisibleForTesting;
 import com.linkedin.pinot.common.config.IndexingConfig;
 import com.linkedin.pinot.common.config.TableConfig;
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
 import com.linkedin.pinot.common.segment.ReadMode;
 import com.linkedin.pinot.core.data.manager.config.InstanceDataManagerConfig;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
@@ -45,6 +46,7 @@ public class IndexLoadingConfig {
   private Map<String, String> _noDictionaryConfig = new HashMap<>();
   private Set<String> _onHeapDictionaryColumns = new HashSet<>();
   private Set<String> _bloomFilterColumns = new HashSet<>();
+  private TextSearchIndexConfig _textSearchIndexConfig;
 
   private SegmentVersion _segmentVersion;
   // This value will remain true only when the empty constructor is invoked.
@@ -108,6 +110,8 @@ public class IndexLoadingConfig {
       _columnMinMaxValueGeneratorMode =
           ColumnMinMaxValueGeneratorMode.valueOf(columnMinMaxValueGeneratorMode.toUpperCase());
     }
+
+    _textSearchIndexConfig = indexingConfig.getTextSearchIndexConfig();
   }
 
   private void extractFromInstanceConfig(@Nonnull InstanceDataManagerConfig instanceDataManagerConfig) {
@@ -205,6 +209,10 @@ public class IndexLoadingConfig {
     return _segmentVersion;
   }
 
+  public TextSearchIndexConfig getTextSearchIndexConfig() {
+    return _textSearchIndexConfig;
+  }
+
   /**
    * For tests only.
    */
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/invertedindex/InvertedIndexHandler.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/invertedindex/InvertedIndexHandler.java
index 993087a..e7f4ccd 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/invertedindex/InvertedIndexHandler.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/loader/invertedindex/InvertedIndexHandler.java
@@ -15,13 +15,17 @@
  */
 package com.linkedin.pinot.core.segment.index.loader.invertedindex;
 
+import com.linkedin.pinot.common.config.TextSearchIndexConfig;
+import com.linkedin.pinot.common.data.FieldSpec;
 import com.linkedin.pinot.core.indexsegment.generator.SegmentVersion;
 import com.linkedin.pinot.core.io.reader.DataFileReader;
 import com.linkedin.pinot.core.io.reader.SingleColumnMultiValueReader;
 import com.linkedin.pinot.core.io.reader.impl.v1.FixedBitMultiValueReader;
 import com.linkedin.pinot.core.io.reader.impl.v1.FixedBitSingleValueReader;
+import com.linkedin.pinot.core.segment.creator.NoDictionaryBasedInvertedIndexCreator;
 import com.linkedin.pinot.core.segment.creator.impl.V1Constants;
 import com.linkedin.pinot.core.segment.creator.impl.inv.OffHeapBitmapInvertedIndexCreator;
+import com.linkedin.pinot.core.segment.creator.impl.textsearch.TextSearchIndexCreatorFactory;
 import com.linkedin.pinot.core.segment.index.ColumnMetadata;
 import com.linkedin.pinot.core.segment.index.SegmentMetadataImpl;
 import com.linkedin.pinot.core.segment.index.loader.IndexLoadingConfig;
@@ -43,6 +47,7 @@ public class InvertedIndexHandler {
   private static final Logger LOGGER = LoggerFactory.getLogger(InvertedIndexHandler.class);
 
   private final File _indexDir;
+  private final IndexLoadingConfig _indexLoadingConfig;
   private final SegmentDirectory.Writer _segmentWriter;
   private final String _segmentName;
   private final SegmentVersion _segmentVersion;
@@ -51,6 +56,7 @@ public class InvertedIndexHandler {
   public InvertedIndexHandler(@Nonnull File indexDir, @Nonnull SegmentMetadataImpl segmentMetadata,
       @Nonnull IndexLoadingConfig indexLoadingConfig, @Nonnull SegmentDirectory.Writer segmentWriter) {
     _indexDir = indexDir;
+    _indexLoadingConfig = indexLoadingConfig;
     _segmentWriter = segmentWriter;
     _segmentName = segmentMetadata.getName();
     _segmentVersion = SegmentVersion.valueOf(segmentMetadata.getVersion());
@@ -65,8 +71,20 @@ public class InvertedIndexHandler {
   }
 
   public void createInvertedIndices() throws IOException {
+    TextSearchIndexConfig textSearchConfig = _indexLoadingConfig.getTextSearchIndexConfig();
+    if (textSearchConfig == null) {
+      textSearchConfig = TextSearchIndexConfig.getDefaultConfig();
+    }
     for (ColumnMetadata columnMetadata : _invertedIndexColumns) {
-      createInvertedIndexForColumn(columnMetadata);
+      if (columnMetadata.getDataType() == FieldSpec.DataType.TEXT) {
+        NoDictionaryBasedInvertedIndexCreator indexer =
+            TextSearchIndexCreatorFactory.createSearchIndexer(textSearchConfig, columnMetadata.getColumnName(), _indexDir);
+        if (!indexer.indexExists(columnMetadata.getColumnName())) {
+          // create index
+        }
+      } else {
+        createInvertedIndexForColumn(columnMetadata);
+      }
     }
   }
 
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/LuceneSearchIndexReader.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/LuceneSearchIndexReader.java
new file mode 100644
index 0000000..32e762e
--- /dev/null
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/LuceneSearchIndexReader.java
@@ -0,0 +1,63 @@
+package com.linkedin.pinot.core.segment.index.readers;
+
+import com.linkedin.pinot.core.segment.creator.impl.textsearch.LuceneIndexCreator;
+import java.io.File;
+import java.io.IOException;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class LuceneSearchIndexReader implements SearchIndexReader<TopDocs> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(LuceneSearchIndexReader.class);
+  private final IndexSearcher _searcher;
+  private final Analyzer _analyzer = new StandardAnalyzer();
+
+  public LuceneSearchIndexReader(String columnName, File segmentIndexDir) {
+
+    try {
+      File searchIndexDir = new File(segmentIndexDir.getPath() + "/" + columnName);
+      Directory index = FSDirectory.open(searchIndexDir.toPath());
+      IndexReader reader = DirectoryReader.open(index);
+      _searcher = new IndexSearcher(reader);
+    } catch (IOException e) {
+      LOGGER.error("Encountered error creating LuceneSearchIndexReader ", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public TopDocs getDocIds(String queryStr, String options) {
+    QueryParser queryParser = new QueryParser(LuceneIndexCreator.DEFAULT_FIELD, _analyzer);
+    Query query = null;
+    try {
+      query = queryParser.parse(queryStr);
+    } catch (ParseException e) {
+      LOGGER.error("Encountered exception while parsing query {}", queryStr, e);
+      throw new RuntimeException(e);
+    }
+    TopDocs docs = null;
+    try {
+      docs = _searcher.search(query, Integer.MAX_VALUE);
+    } catch (IOException e) {
+      LOGGER.error("Encountered exception while executing search query {}", queryStr, e);
+      throw new RuntimeException(e);
+    }
+    return docs;
+  }
+
+  @Override
+  public void close() throws IOException {
+  }
+}
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/SearchIndexReader.java
similarity index 72%
copy from pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java
copy to pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/SearchIndexReader.java
index a91c7d8..e575cb6 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/creator/NoDictionaryBasedInvertedIndexCreator.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/index/readers/SearchIndexReader.java
@@ -13,15 +13,15 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package com.linkedin.pinot.core.segment.creator;
+package com.linkedin.pinot.core.segment.index.readers;
 
-/**
- * A No-Dictionary based InvertedIndexCreator.
- */
-public interface NoDictionaryBasedInvertedIndexCreator extends InvertedIndexCreator {
+import java.io.Closeable;
+
+
+public interface SearchIndexReader<T> extends Closeable {
 
   /**
-   * Add a document field to the index
+   * Get the document ids for the given search query.
    */
-  void add(Object doc);
+  T getDocIds(String query, String options);
 }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/virtualcolumn/VirtualColumnIndexContainer.java b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/virtualcolumn/VirtualColumnIndexContainer.java
index f3d1c52..9af4242 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/segment/virtualcolumn/VirtualColumnIndexContainer.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/segment/virtualcolumn/VirtualColumnIndexContainer.java
@@ -21,6 +21,7 @@ import com.linkedin.pinot.core.segment.index.column.PhysicalColumnIndexContainer
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 
 
 /**
@@ -49,6 +50,11 @@ public class VirtualColumnIndexContainer implements ColumnIndexContainer {
   }
 
   @Override
+  public SearchIndexReader getSearchIndex() {
+    return null;
+  }
+
+  @Override
   public Dictionary getDictionary() {
     return _dictionary;
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeDimensionDataSource.java b/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeDimensionDataSource.java
index 8959969..7c95501 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeDimensionDataSource.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeDimensionDataSource.java
@@ -24,6 +24,7 @@ import com.linkedin.pinot.core.operator.blocks.SingleValueBlock;
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 import com.linkedin.pinot.core.segment.memory.PinotDataBuffer;
 
 
@@ -77,6 +78,11 @@ public class StarTreeDimensionDataSource extends DataSource {
       }
 
       @Override
+      public boolean hasSearchIndex() {
+        return false;
+      }
+
+      @Override
       public boolean hasDictionary() {
         return true;
       }
@@ -101,6 +107,11 @@ public class StarTreeDimensionDataSource extends DataSource {
   }
 
   @Override
+  public SearchIndexReader getSearchIndex() {
+    return null;
+  }
+
+  @Override
   public BloomFilterReader getBloomFilter() {
     return null;
   }
diff --git a/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeMetricDataSource.java b/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeMetricDataSource.java
index f9ac54d..e09fd8a 100644
--- a/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeMetricDataSource.java
+++ b/pinot-core/src/main/java/com/linkedin/pinot/core/startree/v2/store/StarTreeMetricDataSource.java
@@ -27,6 +27,7 @@ import com.linkedin.pinot.core.operator.blocks.SingleValueBlock;
 import com.linkedin.pinot.core.segment.index.readers.BloomFilterReader;
 import com.linkedin.pinot.core.segment.index.readers.Dictionary;
 import com.linkedin.pinot.core.segment.index.readers.InvertedIndexReader;
+import com.linkedin.pinot.core.segment.index.readers.SearchIndexReader;
 import com.linkedin.pinot.core.segment.memory.PinotDataBuffer;
 
 
@@ -82,6 +83,11 @@ public class StarTreeMetricDataSource extends DataSource {
       }
 
       @Override
+      public boolean hasSearchIndex() {
+        return false;
+      }
+
+      @Override
       public boolean hasDictionary() {
         return false;
       }
@@ -106,6 +112,11 @@ public class StarTreeMetricDataSource extends DataSource {
   }
 
   @Override
+  public SearchIndexReader getSearchIndex() {
+    return null;
+  }
+
+  @Override
   public Dictionary getDictionary() {
     return null;
   }
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/core/common/RealtimeNoDictionaryTest.java b/pinot-core/src/test/java/com/linkedin/pinot/core/common/RealtimeNoDictionaryTest.java
index 52b8542..8fd4118 100644
--- a/pinot-core/src/test/java/com/linkedin/pinot/core/common/RealtimeNoDictionaryTest.java
+++ b/pinot-core/src/test/java/com/linkedin/pinot/core/common/RealtimeNoDictionaryTest.java
@@ -86,10 +86,10 @@ public class RealtimeNoDictionaryTest {
     }
 
     Map<String, DataSource> dataSourceBlock = new HashMap<>();
-    dataSourceBlock.put(INT_COL_NAME, new ColumnDataSource(intSpec, NUM_ROWS, 0, intRawIndex, null, null, null));
-    dataSourceBlock.put(LONG_COL_NAME, new ColumnDataSource(longSpec, NUM_ROWS, 0, longRawIndex, null, null, null));
-    dataSourceBlock.put(FLOAT_COL_NAME, new ColumnDataSource(floatSpec, NUM_ROWS, 0, floatRawIndex, null, null, null));
-    dataSourceBlock.put(DOUBLE_COL_NAME, new ColumnDataSource(doubleSpec, NUM_ROWS, 0, doubleRawIndex, null, null, null));
+    dataSourceBlock.put(INT_COL_NAME, new ColumnDataSource(intSpec, NUM_ROWS, 0, intRawIndex, null, null, null, null));
+    dataSourceBlock.put(LONG_COL_NAME, new ColumnDataSource(longSpec, NUM_ROWS, 0, longRawIndex, null, null, null, null));
+    dataSourceBlock.put(FLOAT_COL_NAME, new ColumnDataSource(floatSpec, NUM_ROWS, 0, floatRawIndex, null, null, null, null));
+    dataSourceBlock.put(DOUBLE_COL_NAME, new ColumnDataSource(doubleSpec, NUM_ROWS, 0, doubleRawIndex, null, null, null, null));
 
     return new DataFetcher(dataSourceBlock);
   }
diff --git a/pinot-core/src/test/java/com/linkedin/pinot/queries/TextMatchQueriesTest.java b/pinot-core/src/test/java/com/linkedin/pinot/queries/TextMatchQueriesTest.java
new file mode 100644
index 0000000..7e77ac7
--- /dev/null
+++ b/pinot-core/src/test/java/com/linkedin/pinot/queries/TextMatchQueriesTest.java
@@ -0,0 +1,188 @@
+/**
+ * Copyright (C) 2014-2018 LinkedIn Corp. (pinot-core@linkedin.com)
+ *
+ * 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 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 com.linkedin.pinot.queries;
+
+import com.linkedin.pinot.common.data.DimensionFieldSpec;
+import com.linkedin.pinot.common.data.FieldSpec;
+import com.linkedin.pinot.common.data.MetricFieldSpec;
+import com.linkedin.pinot.common.data.Schema;
+import com.linkedin.pinot.common.response.broker.AggregationResult;
+import com.linkedin.pinot.common.response.broker.BrokerResponseNative;
+import com.linkedin.pinot.common.response.broker.GroupByResult;
+import com.linkedin.pinot.common.segment.ReadMode;
+import com.linkedin.pinot.core.data.GenericRow;
+import com.linkedin.pinot.core.data.manager.SegmentDataManager;
+import com.linkedin.pinot.core.data.manager.offline.ImmutableSegmentDataManager;
+import com.linkedin.pinot.core.data.readers.GenericRowRecordReader;
+import com.linkedin.pinot.core.data.readers.RecordReader;
+import com.linkedin.pinot.core.indexsegment.IndexSegment;
+import com.linkedin.pinot.core.indexsegment.generator.SegmentGeneratorConfig;
+import com.linkedin.pinot.core.indexsegment.immutable.ImmutableSegment;
+import com.linkedin.pinot.core.indexsegment.immutable.ImmutableSegmentLoader;
+import com.linkedin.pinot.core.operator.blocks.IntermediateResultsBlock;
+import com.linkedin.pinot.core.operator.query.AggregationGroupByOperator;
+import com.linkedin.pinot.core.operator.query.AggregationOperator;
+import com.linkedin.pinot.core.query.aggregation.function.PercentileTDigestAggregationFunction;
+import com.linkedin.pinot.core.query.aggregation.groupby.AggregationGroupByResult;
+import com.linkedin.pinot.core.query.aggregation.groupby.GroupKeyGenerator;
+import com.linkedin.pinot.core.segment.creator.impl.SegmentIndexCreationDriverImpl;
+import com.linkedin.pinot.core.segment.creator.impl.textsearch.LuceneIndexCreator;
+import com.linkedin.pinot.core.segment.index.loader.IndexLoadingConfig;
+import com.tdunning.math.stats.TDigest;
+import it.unimi.dsi.fastutil.doubles.DoubleList;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.commons.io.FileUtils;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+
+/**
+ * Tests for TEXT_MATCH queries.
+ *
+ * <ul>
+ *   <li>Generates a segment with a TEXT column, and 2 group by columns.</li>
+ *   <li>Runs simple text-match queries.</li>
+ * </ul>
+ */
+public class TextMatchQueriesTest extends BaseQueriesTest {
+  private static final File INDEX_DIR = new File(FileUtils.getTempDirectory(), "TextMatchQueriesTest");
+  private static final String TABLE_NAME = "testTable";
+  private static final String SEGMENT_NAME = "testSegment";
+
+  private static final int NUM_ROWS = 2000;
+  private static final double VALUE_RANGE = Integer.MAX_VALUE;
+  private static final String TEXT_COLUMN = "textColumn";
+  private static final String DOUBLE_COLUMN = "doubleColumn";
+  private static final String GROUP_BY_COLUMN = "groupByColumn";
+  private static final String[] GROUPS = new String[]{"G1", "G2", "G3"};
+  private static final long RANDOM_SEED = System.nanoTime();
+  private static final Random RANDOM = new Random(RANDOM_SEED);
+  private static final String ERROR_MESSAGE = "Random seed: " + RANDOM_SEED;
+
+  private static final String QUERY_FILTER = "WHERE TEXT_MATCH(textColumn, \"Fire*\", \"\")";
+
+  private ImmutableSegment _indexSegment;
+  private List<SegmentDataManager> _segmentDataManagers;
+
+  @Override
+  protected String getFilter() {
+    return QUERY_FILTER;
+  }
+
+  @Override
+  protected IndexSegment getIndexSegment() {
+    return _indexSegment;
+  }
+
+  @Override
+  protected List<SegmentDataManager> getSegmentDataManagers() {
+    return _segmentDataManagers;
+  }
+
+  @BeforeClass
+  public void setUp() throws Exception {
+    FileUtils.deleteQuietly(INDEX_DIR);
+
+    try {
+      buildSegment();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+    IndexLoadingConfig config = new IndexLoadingConfig();
+    config.setInvertedIndexColumns(Collections.singleton(TEXT_COLUMN));
+    try {
+      _indexSegment = ImmutableSegmentLoader.load(new File(INDEX_DIR, SEGMENT_NAME), config);
+    } catch(Exception e) {
+      e.printStackTrace();
+    }
+    _segmentDataManagers =
+        Arrays.asList(new ImmutableSegmentDataManager(_indexSegment), new ImmutableSegmentDataManager(_indexSegment));
+  }
+
+  protected void buildSegment() throws Exception {
+
+    List<GenericRow> rows = new ArrayList<>(NUM_ROWS);
+    URL resourceUrl = getClass().getClassLoader().getResource("data/logfile");
+    File file = new File(resourceUrl.getFile());
+    try (BufferedReader br = new BufferedReader(new FileReader(file))) {
+
+      for (int i = 0; i < NUM_ROWS; i++) {
+        HashMap<String, Object> valueMap = new HashMap<>();
+
+        double value = RANDOM.nextDouble() * VALUE_RANGE;
+        valueMap.put(DOUBLE_COLUMN, value);
+
+        // add the text column
+        valueMap.put(TEXT_COLUMN, br.readLine());
+
+        String group = GROUPS[RANDOM.nextInt(GROUPS.length)];
+        valueMap.put(GROUP_BY_COLUMN, group);
+
+        GenericRow genericRow = new GenericRow();
+        genericRow.init(valueMap);
+        rows.add(genericRow);
+      }
+    }
+
+    Schema schema = new Schema();
+    schema.addField(new MetricFieldSpec(DOUBLE_COLUMN, FieldSpec.DataType.DOUBLE));
+    schema.addField(new DimensionFieldSpec(TEXT_COLUMN, FieldSpec.DataType.TEXT, true));
+    schema.addField(new DimensionFieldSpec(GROUP_BY_COLUMN, FieldSpec.DataType.STRING, true));
+
+    SegmentGeneratorConfig config = new SegmentGeneratorConfig(schema);
+    config.setOutDir(INDEX_DIR.getPath());
+    config.setTableName(TABLE_NAME);
+    config.setSegmentName(SEGMENT_NAME);
+    config.setInvertedIndexCreationColumns(Collections.singletonList(TEXT_COLUMN));
+    config.setRawIndexCreationColumns(Collections.singletonList(TEXT_COLUMN));
+
+    SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl();
+    try (RecordReader recordReader = new GenericRowRecordReader(rows, schema)) {
+      driver.init(config, recordReader);
+      driver.build();
+    }
+  }
+
+  @Test
+  public void testTextSearchFilter() {
+    String query = "SELECT COUNT(*) FROM testTable " + getFilter();
+
+    BrokerResponseNative brokerResponse = getBrokerResponseForQuery(query);
+    // todo: verify counts here
+    QueriesTestUtils.testInterSegmentAggregationResult(brokerResponse, 3364, 0L, 0L, 8000,
+        new String[]{"3364"});
+  }
+
+  @AfterClass
+  public void tearDown() {
+    _indexSegment.destroy();
+    FileUtils.deleteQuietly(INDEX_DIR);
+  }
+}
diff --git a/pinot-core/src/test/resources/data/logfile b/pinot-core/src/test/resources/data/logfile
new file mode 100644
index 0000000..4f65bb1
--- /dev/null
+++ b/pinot-core/src/test/resources/data/logfile
@@ -0,0 +1,1000 @@
+
+109.169.248.247 - - [12/Dec/2015:18:25:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.169.248.247 - - [12/Dec/2015:18:25:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.177.4 - - [12/Dec/2015:18:31:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.177.4 - - [12/Dec/2015:18:31:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:18:31:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:18:31:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.198.15 - - [12/Dec/2015:18:32:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.198.15 - - [12/Dec/2015:18:32:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.184.11.34 - - [12/Dec/2015:18:32:56 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.184.11.34 - - [12/Dec/2015:18:32:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.29.79 - - [12/Dec/2015:18:33:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.29.79 - - [12/Dec/2015:18:33:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+90.154.66.233 - - [12/Dec/2015:18:36:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+90.154.66.233 - - [12/Dec/2015:18:36:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [12/Dec/2015:18:38:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [12/Dec/2015:18:38:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.188.245.16 - - [12/Dec/2015:18:38:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.188.245.16 - - [12/Dec/2015:18:38:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.213.133 - - [12/Dec/2015:18:39:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.213.133 - - [12/Dec/2015:18:39:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.119.130 - - [12/Dec/2015:18:40:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.119.130 - - [12/Dec/2015:18:40:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.0.41 - - [12/Dec/2015:18:44:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.0.41 - - [12/Dec/2015:18:44:48 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.252.165.97 - - [12/Dec/2015:18:44:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.252.165.97 - - [12/Dec/2015:18:44:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.202.233.212 - - [12/Dec/2015:18:46:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.202.233.212 - - [12/Dec/2015:18:46:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.219.151.232 - - [12/Dec/2015:18:46:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.219.151.232 - - [12/Dec/2015:18:46:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.184.174 - - [12/Dec/2015:18:51:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.184.174 - - [12/Dec/2015:18:51:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.116.138.82 - - [12/Dec/2015:18:51:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.116.138.82 - - [12/Dec/2015:18:51:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.47.189.163 - - [12/Dec/2015:18:53:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.47.189.163 - - [12/Dec/2015:18:53:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.144.231 - - [12/Dec/2015:18:57:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.144.231 - - [12/Dec/2015:18:57:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.134.32 - - [12/Dec/2015:18:58:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.134.32 - - [12/Dec/2015:18:58:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:35 +0100] "GET / HTTP/1.1" 200 10439 "http://top1-seo-service.com/try.php?u=http://almhuette-raith.at" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:35 +0100] "GET /modules/mod_bowslideshow/tmpl/css/bowslideshow.css HTTP/1.1" 200 1725 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/_system/css/general.css HTTP/1.1" 404 239 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /media/system/js/caption.js HTTP/1.1" 200 1963 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/jp_hotel/css/template.css HTTP/1.1" 200 10004 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/jp_hotel/css/layout.css HTTP/1.1" 200 1801 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/jp_hotel/css/menu.css HTTP/1.1" 200 1457 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/jp_hotel/css/suckerfish.css HTTP/1.1" 200 3465 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /templates/jp_hotel/js/moomenu.js HTTP/1.1" 200 4890 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /modules/mod_bowslideshow/tmpl/js/sliderman.1.3.0.js HTTP/1.1" 200 33472 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:36 +0100] "GET /media/system/js/mootools.js HTTP/1.1" 200 74434 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:38 +0100] "GET /images/stories/slideshow/almhuette_raith_01.jpg HTTP/1.1" 200 88161 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:38 +0100] "GET /images/stories/slideshow/almhuette_raith_02.jpg HTTP/1.1" 200 62918 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:38 +0100] "GET /images/stories/slideshow/almhuette_raith_03.jpg HTTP/1.1" 200 87782 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:38 +0100] "GET /images/stories/slideshow/almhuette_raith_06.jpg HTTP/1.1" 200 68977 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/slideshow/almhuette_raith_04.jpg HTTP/1.1" 200 80637 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/slideshow/almhuette_raith_05.jpg HTTP/1.1" 200 77796 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /templates/jp_hotel/images/logo.jpg HTTP/1.1" 200 369 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/slideshow/almhuette_raith_07.jpg HTTP/1.1" 200 94861 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/raith/almhuette_raith.jpg HTTP/1.1" 200 43300 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/raith/wohnraum.jpg HTTP/1.1" 200 43586 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/raith/grillplatz.jpg HTTP/1.1" 200 55303 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:39 +0100] "GET /images/stories/raith/almenland_logo.jpg HTTP/1.1" 200 21490 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /images/stories/raith/oststeiermark.png HTTP/1.1" 200 65225 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /images/stories/raith/garage.jpg HTTP/1.1" 200 57339 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /images/stories/raith/steiermark_herz.png HTTP/1.1" 200 39683 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /modules/mod_bowslideshow/tmpl/images/image_shadow.png HTTP/1.1" 200 5017 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /templates/jp_hotel/images/content_heading.gif HTTP/1.1" 200 69 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /images/bg_raith.jpg HTTP/1.1" 200 329961 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:40 +0100] "GET /templates/jp_hotel/images/module_heading.gif HTTP/1.1" 200 83 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_02.jpg HTTP/1.1" 200 62918 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_04.jpg HTTP/1.1" 200 80637 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_05.jpg HTTP/1.1" 200 77796 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_06.jpg HTTP/1.1" 200 68977 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_03.jpg HTTP/1.1" 200 87782 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:48 +0100] "GET /images/stories/slideshow/almhuette_raith_01.jpg HTTP/1.1" 200 88161 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+191.182.199.16 - - [12/Dec/2015:19:02:49 +0100] "GET /images/stories/slideshow/almhuette_raith_07.jpg HTTP/1.1" 200 43495 "http://almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/36.0.1985.143 Safari/537.36" "-"
+176.212.187.84 - - [12/Dec/2015:19:03:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.212.187.84 - - [12/Dec/2015:19:03:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+180.180.104.183 - - [12/Dec/2015:19:04:13 +0100] "GET /apache-log/access.log HTTP/1.1" 200 17511 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+176.194.214.104 - - [12/Dec/2015:19:09:36 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.194.214.104 - - [12/Dec/2015:19:09:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.16.1.18 - - [12/Dec/2015:19:12:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.16.1.18 - - [12/Dec/2015:19:12:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.6.184.110 - - [12/Dec/2015:19:17:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.6.184.110 - - [12/Dec/2015:19:17:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.88.222.215 - - [12/Dec/2015:19:20:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.88.222.215 - - [12/Dec/2015:19:20:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.19 - - [12/Dec/2015:19:23:59 +0100] "GET /index.php?option=com_content&view=article&id=46&Itemid=54 HTTP/1.1" 200 8932 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+87.255.246.50 - - [12/Dec/2015:19:24:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [12/Dec/2015:19:24:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.116.249.142 - - [12/Dec/2015:19:28:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.116.249.142 - - [12/Dec/2015:19:28:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.39.229.183 - - [12/Dec/2015:19:29:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.39.229.183 - - [12/Dec/2015:19:29:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [12/Dec/2015:19:31:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [12/Dec/2015:19:31:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.248.235.103 - - [12/Dec/2015:19:31:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.248.235.103 - - [12/Dec/2015:19:31:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:19:37:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:19:37:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.252.11.47 - - [12/Dec/2015:19:38:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.252.11.47 - - [12/Dec/2015:19:38:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.207.163 - - [12/Dec/2015:19:40:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.207.163 - - [12/Dec/2015:19:40:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.34.161.222 - - [12/Dec/2015:19:43:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.34.161.222 - - [12/Dec/2015:19:43:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET / HTTP/1.1" 200 10479 "http://www.almenland.at/almhuetten-mit-naechtigung.html" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /modules/mod_bowslideshow/tmpl/css/bowslideshow.css HTTP/1.1" 200 1725 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/_system/css/general.css HTTP/1.1" 404 239 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/css/template.css HTTP/1.1" 200 10004 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/css/layout.css HTTP/1.1" 200 1801 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/css/menu.css HTTP/1.1" 200 1457 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/css/suckerfish.css HTTP/1.1" 200 3465 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /media/system/js/caption.js HTTP/1.1" 200 1963 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/js/moomenu.js HTTP/1.1" 200 4890 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /modules/mod_bowslideshow/tmpl/js/sliderman.1.3.0.js HTTP/1.1" 200 33472 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /media/system/js/mootools.js HTTP/1.1" 200 74434 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:06 +0100] "GET /templates/jp_hotel/images/logo.jpg HTTP/1.1" 200 369 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:07 +0100] "GET /images/stories/slideshow/almhuette_raith_02.jpg HTTP/1.1" 200 62918 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:07 +0100] "GET /images/stories/raith/wohnraum.jpg HTTP/1.1" 200 43586 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:07 +0100] "GET /images/stories/slideshow/almhuette_raith_01.jpg HTTP/1.1" 200 88161 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:08 +0100] "GET /images/stories/raith/grillplatz.jpg HTTP/1.1" 200 55303 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:08 +0100] "GET /images/stories/raith/garage.jpg HTTP/1.1" 200 57339 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /images/stories/slideshow/almhuette_raith_03.jpg HTTP/1.1" 200 87782 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /images/stories/slideshow/almhuette_raith_06.jpg HTTP/1.1" 200 68977 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /images/stories/raith/almhuette_raith.jpg HTTP/1.1" 200 43300 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /modules/mod_bowslideshow/tmpl/images/image_shadow.png HTTP/1.1" 200 5017 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /templates/jp_hotel/images/content_heading.gif HTTP/1.1" 200 69 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:09 +0100] "GET /templates/jp_hotel/images/module_heading.gif HTTP/1.1" 200 83 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:10 +0100] "GET /images/stories/slideshow/almhuette_raith_05.jpg HTTP/1.1" 200 77796 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:10 +0100] "GET /images/stories/raith/almenland_logo.jpg HTTP/1.1" 200 21490 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:10 +0100] "GET /images/stories/raith/steiermark_herz.png HTTP/1.1" 200 39683 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:10 +0100] "GET /images/stories/slideshow/almhuette_raith_04.jpg HTTP/1.1" 200 80637 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:10 +0100] "GET /images/stories/raith/oststeiermark.png HTTP/1.1" 200 65225 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:12 +0100] "GET /images/stories/slideshow/almhuette_raith_07.jpg HTTP/1.1" 200 94861 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:13 +0100] "GET /images/bg_raith.jpg HTTP/1.1" 200 329961 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.45.108.168 - - [12/Dec/2015:19:44:15 +0100] "GET /favicon.ico HTTP/1.1" 404 217 "-" "Mozilla/5.0 (Linux; Android 4.4.2; de-at; SAMSUNG GT-I9301I Build/KOT49H) AppleWebKit/537.36 (KHTML, like Gecko) Version/1.5 Chrome/28.0.1500.94 Mobile Safari/537.36" "-"
+188.254.126.77 - - [12/Dec/2015:19:46:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.254.126.77 - - [12/Dec/2015:19:46:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.103 - - [12/Dec/2015:19:47:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.103 - - [12/Dec/2015:19:47:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.81.147.127 - - [12/Dec/2015:19:48:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.81.147.127 - - [12/Dec/2015:19:48:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.144.231 - - [12/Dec/2015:19:51:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.144.231 - - [12/Dec/2015:19:51:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+195.91.157.50 - - [12/Dec/2015:19:52:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+195.91.157.50 - - [12/Dec/2015:19:52:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.164.122 - - [12/Dec/2015:19:52:40 +0100] "GET /administrator/ HTTP/1.0" 200 4250 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.164.122 - - [12/Dec/2015:19:52:40 +0100] "POST /administrator/index.php HTTP/1.0" 200 4481 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.112.194.169 - - [12/Dec/2015:19:53:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.112.194.169 - - [12/Dec/2015:19:53:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:19:54:40 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:19:54:40 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.243.115.178 - - [12/Dec/2015:19:55:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.243.115.178 - - [12/Dec/2015:19:55:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:19:56:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:19:56:15 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.45.201.226 - - [12/Dec/2015:19:57:45 +0100] "GET /apache-log/access.log HTTP/1.1" 200 36435 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/39.0.2171.95 Safari/537.36" "-"
+176.213.82.180 - - [12/Dec/2015:20:01:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.213.82.180 - - [12/Dec/2015:20:01:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.173.80.169 - - [12/Dec/2015:20:04:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.173.80.169 - - [12/Dec/2015:20:04:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.132.155 - - [12/Dec/2015:20:06:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.132.155 - - [12/Dec/2015:20:06:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:20:06:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:20:06:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [12/Dec/2015:20:07:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [12/Dec/2015:20:07:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [12/Dec/2015:20:13:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [12/Dec/2015:20:13:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.17.135.224 - - [12/Dec/2015:20:15:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.17.135.224 - - [12/Dec/2015:20:15:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.134.32 - - [12/Dec/2015:20:19:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.134.32 - - [12/Dec/2015:20:19:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+180.76.15.137 - - [12/Dec/2015:20:22:24 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (Windows NT 5.1; rv:6.0.2) Gecko/20100101 Firefox/6.0.2" "-"
+109.252.11.47 - - [12/Dec/2015:20:22:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.252.11.47 - - [12/Dec/2015:20:22:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [12/Dec/2015:20:22:54 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [12/Dec/2015:20:22:54 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.213.82.180 - - [12/Dec/2015:20:24:03 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.213.82.180 - - [12/Dec/2015:20:24:03 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.85.222 - - [12/Dec/2015:20:28:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.85.222 - - [12/Dec/2015:20:28:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.18.95.55 - - [12/Dec/2015:20:28:40 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.18.95.55 - - [12/Dec/2015:20:28:40 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.211.128.249 - - [12/Dec/2015:20:30:41 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.211.128.249 - - [12/Dec/2015:20:30:41 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.45.11.101 - - [12/Dec/2015:20:31:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.45.11.101 - - [12/Dec/2015:20:31:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.252.127.248 - - [12/Dec/2015:20:33:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.252.127.248 - - [12/Dec/2015:20:33:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:20:36:12 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:20:36:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.103 - - [12/Dec/2015:20:36:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.103 - - [12/Dec/2015:20:36:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.5 - - [12/Dec/2015:20:37:57 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+109.252.11.47 - - [12/Dec/2015:20:40:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.252.11.47 - - [12/Dec/2015:20:40:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.244.224.113 - - [12/Dec/2015:20:41:06 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.244.224.113 - - [12/Dec/2015:20:41:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:20:42:54 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:20:42:54 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.161.253 - - [12/Dec/2015:20:46:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.161.253 - - [12/Dec/2015:20:46:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.165 - - [12/Dec/2015:20:48:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.165 - - [12/Dec/2015:20:48:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.17.60.192 - - [12/Dec/2015:20:58:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.17.60.192 - - [12/Dec/2015:20:58:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.172.31.79 - - [12/Dec/2015:20:59:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.172.31.79 - - [12/Dec/2015:20:59:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [12/Dec/2015:21:02:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [12/Dec/2015:21:02:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.16 - - [12/Dec/2015:21:02:37 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+46.72.184.174 - - [12/Dec/2015:21:04:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.184.174 - - [12/Dec/2015:21:04:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.134.205.87 - - [12/Dec/2015:21:06:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.134.205.87 - - [12/Dec/2015:21:06:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [12/Dec/2015:21:09:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [12/Dec/2015:21:09:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.237.109.103 - - [12/Dec/2015:21:11:14 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.237.109.103 - - [12/Dec/2015:21:11:15 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.164.122 - - [12/Dec/2015:21:14:51 +0100] "GET /administrator/ HTTP/1.0" 200 4250 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.164.122 - - [12/Dec/2015:21:14:52 +0100] "POST /administrator/index.php HTTP/1.0" 200 4481 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:21:16:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:21:16:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.117 - - [12/Dec/2015:21:18:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.117 - - [12/Dec/2015:21:18:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+107.168.140.56 - - [12/Dec/2015:21:19:54 +0100] "GET /apache-log/access.log HTTP/1.0" 200 49541 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+83.167.113.100 - - [12/Dec/2015:21:24:36 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:21:24:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.8 - - [12/Dec/2015:21:26:04 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.8 - - [12/Dec/2015:21:26:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:21:27:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.147.245 - - [12/Dec/2015:21:27:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.164.167.182 - - [12/Dec/2015:21:28:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.164.167.182 - - [12/Dec/2015:21:28:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.105.39.25 - - [12/Dec/2015:21:31:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.105.39.25 - - [12/Dec/2015:21:31:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [12/Dec/2015:21:34:09 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [12/Dec/2015:21:34:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+41.33.191.114 - - [12/Dec/2015:21:35:59 +0100] "GET /apache-log/access.log HTTP/1.1" 200 52053 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+46.61.15.146 - - [12/Dec/2015:21:36:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.15.146 - - [12/Dec/2015:21:36:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.159.214.230 - - [12/Dec/2015:21:42:48 +0100] "GET / HTTP/1.1" 200 10479 "http://www.bigfinder.de/?c=ws&q=start&v=1" "T-Online Browser (Windows NT 6.1; de)" "-"
+84.18.126.216 - - [12/Dec/2015:21:43:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+84.18.126.216 - - [12/Dec/2015:21:43:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.123.230.195 - - [12/Dec/2015:21:47:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.123.230.195 - - [12/Dec/2015:21:47:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.16 - - [12/Dec/2015:21:48:18 +0100] "GET /index.php?option=com_content&view=article&id=46&Itemid=54 HTTP/1.1" 200 8938 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+89.105.140.247 - - [12/Dec/2015:21:48:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.105.140.247 - - [12/Dec/2015:21:48:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [12/Dec/2015:21:48:54 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [12/Dec/2015:21:48:54 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.41.213.58 - - [12/Dec/2015:21:50:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.41.213.58 - - [12/Dec/2015:21:50:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.244.224.113 - - [12/Dec/2015:21:51:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.244.224.113 - - [12/Dec/2015:21:51:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [12/Dec/2015:21:53:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [12/Dec/2015:21:53:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.7.30 - - [12/Dec/2015:21:54:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.7.30 - - [12/Dec/2015:21:54:15 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+159.182.107.1 - - [12/Dec/2015:21:56:50 +0100] "GET /apache-log/access.log HTTP/1.0" 200 56070 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/39.0.2171.95 Safari/537.36" "-"
+91.227.190.44 - - [12/Dec/2015:21:59:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.190.44 - - [12/Dec/2015:21:59:33 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.117 - - [12/Dec/2015:22:02:03 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.117 - - [12/Dec/2015:22:02:03 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [12/Dec/2015:22:02:47 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [12/Dec/2015:22:02:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.190.44 - - [12/Dec/2015:22:14:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.190.44 - - [12/Dec/2015:22:14:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.164.219 - - [12/Dec/2015:22:15:38 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.164.219 - - [12/Dec/2015:22:15:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.58.71.107 - - [12/Dec/2015:22:16:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.58.71.107 - - [12/Dec/2015:22:16:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.135.72 - - [12/Dec/2015:22:19:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.135.72 - - [12/Dec/2015:22:19:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.101.154.122 - - [12/Dec/2015:22:20:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.101.154.122 - - [12/Dec/2015:22:20:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.70.41.172 - - [12/Dec/2015:22:22:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.70.41.172 - - [12/Dec/2015:22:22:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.165 - - [12/Dec/2015:22:22:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.165 - - [12/Dec/2015:22:22:15 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.172.1 - - [12/Dec/2015:22:22:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.172.1 - - [12/Dec/2015:22:22:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.34.203 - - [12/Dec/2015:22:24:32 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.34.203 - - [12/Dec/2015:22:24:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.201.52.125 - - [12/Dec/2015:22:26:32 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.201.52.125 - - [12/Dec/2015:22:26:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.192.246.119 - - [12/Dec/2015:22:27:03 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.192.246.119 - - [12/Dec/2015:22:27:03 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:22:28:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:22:28:10 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.28.162.61 - - [12/Dec/2015:22:28:47 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.28.162.61 - - [12/Dec/2015:22:28:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.172.31.79 - - [12/Dec/2015:22:29:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.172.31.79 - - [12/Dec/2015:22:29:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.230.81.36 - - [12/Dec/2015:22:31:36 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.230.81.36 - - [12/Dec/2015:22:31:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+192.227.224.48 - - [12/Dec/2015:22:37:42 +0100] "GET /apache-log/access.log HTTP/1.0" 200 63067 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Iron/29.0.1600.1 Chrome/29.0.1600.1 Safari/537.36" "-"
+85.174.39.161 - - [12/Dec/2015:22:38:43 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.174.39.161 - - [12/Dec/2015:22:38:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.172.1 - - [12/Dec/2015:22:39:14 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.172.1 - - [12/Dec/2015:22:39:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.173.87.93 - - [12/Dec/2015:22:47:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.173.87.93 - - [12/Dec/2015:22:47:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.78.52.106 - - [12/Dec/2015:22:50:07 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.78.52.106 - - [12/Dec/2015:22:50:07 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [12/Dec/2015:22:51:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [12/Dec/2015:22:51:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.123.230.195 - - [12/Dec/2015:22:53:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.123.230.195 - - [12/Dec/2015:22:53:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:22:54:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [12/Dec/2015:22:54:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.86 - - [12/Dec/2015:22:55:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.61.242.86 - - [12/Dec/2015:22:55:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.157.235.169 - - [12/Dec/2015:23:00:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.157.235.169 - - [12/Dec/2015:23:00:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.48 - - [12/Dec/2015:23:02:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.48 - - [12/Dec/2015:23:02:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:23:05:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [12/Dec/2015:23:05:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.115.96.41 - - [12/Dec/2015:23:07:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.115.96.41 - - [12/Dec/2015:23:07:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+192.162.27.227 - - [12/Dec/2015:23:13:50 +0100] "GET /apache-log/access.log HTTP/1.0" 200 67817 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Iron/29.0.1600.1 Chrome/29.0.1600.1 Safari/537.36" "-"
+109.106.143.62 - - [12/Dec/2015:23:18:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [12/Dec/2015:23:18:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.243.115.178 - - [12/Dec/2015:23:19:23 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.243.115.178 - - [12/Dec/2015:23:19:23 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.5 - - [12/Dec/2015:23:24:28 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+80.66.158.12 - - [12/Dec/2015:23:28:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.12 - - [12/Dec/2015:23:28:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:23:32:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [12/Dec/2015:23:32:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.7.217.50 - - [12/Dec/2015:23:34:36 +0100] "GET /administrator/ HTTP/1.1" 200 4267 "-" "Mozilla/5.0 (X11; Linux i686) AppleWebKit/534.23 (KHTML, like Gecko) Chrome/11.0.686.3 Safari/534.23" "-"
+212.7.217.50 - - [12/Dec/2015:23:34:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4498 "http://www.almhuette-raith.at/administrator/" "Mozilla/5.0 (X11; Linux i686) AppleWebKit/534.23 (KHTML, like Gecko) Chrome/11.0.686.3 Safari/534.23" "-"
+91.233.165.63 - - [12/Dec/2015:23:39:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.233.165.63 - - [12/Dec/2015:23:39:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.47.158.158 - - [12/Dec/2015:23:39:23 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.47.158.158 - - [12/Dec/2015:23:39:23 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [12/Dec/2015:23:43:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [12/Dec/2015:23:43:54 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.248.180.73 - - [12/Dec/2015:23:45:46 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.248.180.73 - - [12/Dec/2015:23:45:46 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.88.155.223 - - [12/Dec/2015:23:47:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.88.155.223 - - [12/Dec/2015:23:47:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.19.16.84 - - [12/Dec/2015:23:52:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.19.16.84 - - [12/Dec/2015:23:52:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [12/Dec/2015:23:53:30 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [12/Dec/2015:23:53:30 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [12/Dec/2015:23:54:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [12/Dec/2015:23:54:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+217.23.187.201 - - [12/Dec/2015:23:58:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+217.23.187.201 - - [12/Dec/2015:23:58:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [12/Dec/2015:23:59:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [12/Dec/2015:23:59:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [12/Dec/2015:23:59:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [12/Dec/2015:23:59:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:00:00:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:00:00:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:00:01:09 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:00:01:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.143.232.11 - - [13/Dec/2015:00:01:23 +0100] "GET /apache-log/access.log HTTP/1.1" 200 75060 "-" "Mozilla/4.0 (compatible; MSIE 6.0; Windows NT 5.1; SV1)" "-"
+2.93.207.163 - - [13/Dec/2015:00:06:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.207.163 - - [13/Dec/2015:00:06:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.191.144.222 - - [13/Dec/2015:00:10:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.191.144.222 - - [13/Dec/2015:00:10:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.229.199.230 - - [13/Dec/2015:00:28:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.229.199.230 - - [13/Dec/2015:00:28:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.48 - - [13/Dec/2015:00:30:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.48 - - [13/Dec/2015:00:30:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [13/Dec/2015:00:35:23 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [13/Dec/2015:00:35:23 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.98 - - [13/Dec/2015:00:37:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.132.98 - - [13/Dec/2015:00:37:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:00:39:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:00:39:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:00:39:46 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:00:39:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.19 - - [13/Dec/2015:00:40:24 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+157.55.39.3 - - [13/Dec/2015:00:46:58 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+92.37.231.110 - - [13/Dec/2015:00:50:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.37.231.110 - - [13/Dec/2015:00:50:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:00:50:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:00:50:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.115.96.41 - - [13/Dec/2015:00:56:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.115.96.41 - - [13/Dec/2015:00:56:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.19.16.84 - - [13/Dec/2015:00:57:32 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.19.16.84 - - [13/Dec/2015:00:57:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+157.55.39.3 - - [13/Dec/2015:01:00:50 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+157.55.39.3 - - [13/Dec/2015:01:01:19 +0100] "GET /icons/text.gif HTTP/1.1" 404 220 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+199.168.148.146 - - [13/Dec/2015:01:01:58 +0100] "GET /apache-log/access.log HTTP/1.0" 200 80346 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+217.23.186.4 - - [13/Dec/2015:01:08:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+217.23.186.4 - - [13/Dec/2015:01:08:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.32.85.147 - - [13/Dec/2015:01:13:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.32.85.147 - - [13/Dec/2015:01:13:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+162.254.33.118 - - [13/Dec/2015:01:29:36 +0100] "GET /apache-log/access.log HTTP/1.0" 200 81357 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Iron/29.0.1600.1 Chrome/29.0.1600.1 Safari/537.36" "-"
+177.98.255.56 - - [13/Dec/2015:01:29:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:01:29:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+23.254.240.222 - - [13/Dec/2015:01:29:58 +0100] "GET /apache-log/access.log HTTP/1.1" 200 81992 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Iron/29.0.1600.1 Chrome/29.0.1600.1 Safari/537.36" "-"
+94.233.8.101 - - [13/Dec/2015:01:33:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.8.101 - - [13/Dec/2015:01:33:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.128.6 - - [13/Dec/2015:01:36:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.128.6 - - [13/Dec/2015:01:36:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:02:30:28 +0100] "GET /administrator/ HTTP/1.1" 500 88 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.197.194.109 - - [13/Dec/2015:02:33:01 +0100] "GET /administrator/ HTTP/1.1" 500 88 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.188.126.7 - - [13/Dec/2015:02:43:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.188.126.7 - - [13/Dec/2015:02:43:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [13/Dec/2015:02:51:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [13/Dec/2015:02:51:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [13/Dec/2015:02:56:23 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.13.146.143 - - [13/Dec/2015:02:56:23 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+181.214.37.170 - - [13/Dec/2015:02:59:39 +0100] "GET /apache-log/access.log HTTP/1.0" 200 84452 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+178.88.155.223 - - [13/Dec/2015:03:02:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.88.155.223 - - [13/Dec/2015:03:02:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.16 - - [13/Dec/2015:03:04:46 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+188.0.189.69 - - [13/Dec/2015:03:04:56 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.0.189.69 - - [13/Dec/2015:03:04:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.186.202.222 - - [13/Dec/2015:03:10:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.186.202.222 - - [13/Dec/2015:03:10:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.101.191 - - [13/Dec/2015:03:13:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.101.191 - - [13/Dec/2015:03:13:49 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.8.101 - - [13/Dec/2015:03:14:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.8.101 - - [13/Dec/2015:03:14:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:03:20:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:03:20:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.96.253.187 - - [13/Dec/2015:03:27:59 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.96.253.187 - - [13/Dec/2015:03:27:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:03:28:54 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:03:28:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:03:29:13 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:03:29:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:03:29:13 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+187.112.238.242 - - [13/Dec/2015:03:37:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.112.238.242 - - [13/Dec/2015:03:37:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+146.120.107.98 - - [13/Dec/2015:03:44:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+146.120.107.98 - - [13/Dec/2015:03:44:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:03:46:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:03:46:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.140.5.75 - - [13/Dec/2015:03:50:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.140.5.75 - - [13/Dec/2015:03:50:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.159.102.216 - - [13/Dec/2015:03:53:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.159.102.216 - - [13/Dec/2015:03:53:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.44.172.217 - - [13/Dec/2015:03:53:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.44.172.217 - - [13/Dec/2015:03:53:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.165.15.204 - - [13/Dec/2015:03:55:30 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (compatible; AhrefsBot/5.0; +http://ahrefs.com/robot/)" "-"
+106.3.37.223 - - [13/Dec/2015:03:56:08 +0100] "GET /apache-log/access.log HTTP/1.1" 200 91071 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+213.167.43.54 - - [13/Dec/2015:03:56:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [13/Dec/2015:03:56:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.60.236.215 - - [13/Dec/2015:03:57:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.60.236.215 - - [13/Dec/2015:03:57:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.145.141.195 - - [13/Dec/2015:03:59:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.145.141.195 - - [13/Dec/2015:03:59:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.31.167.111 - - [13/Dec/2015:04:01:06 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.31.167.111 - - [13/Dec/2015:04:01:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.134.53.96 - - [13/Dec/2015:04:08:50 +0100] "GET /apache-log/access.log HTTP/1.1" 200 92839 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+46.44.5.179 - - [13/Dec/2015:04:10:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.44.5.179 - - [13/Dec/2015:04:10:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:04:10:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:04:10:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.188.126.7 - - [13/Dec/2015:04:12:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.188.126.7 - - [13/Dec/2015:04:12:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:04:14:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:04:14:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [13/Dec/2015:04:23:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.12.28.122 - - [13/Dec/2015:04:23:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+198.46.246.102 - - [13/Dec/2015:04:28:21 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/39.0.2171.65 Safari/537.36" "-"
+213.167.43.54 - - [13/Dec/2015:04:40:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [13/Dec/2015:04:40:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:04:43:03 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:04:43:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.48.218.78 - - [13/Dec/2015:04:46:46 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.48.218.78 - - [13/Dec/2015:04:46:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+157.55.39.8 - - [13/Dec/2015:04:55:10 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+157.55.39.8 - - [13/Dec/2015:04:55:12 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (iPhone; CPU iPhone OS 7_0 like Mac OS X) AppleWebKit/537.51.1 (KHTML, like Gecko) Version/7.0 Mobile/11A465 Safari/9537.53 (compatible; bingbot/2.0;  http://www.bing.com/bingbot.htm)" "-"
+5.140.31.161 - - [13/Dec/2015:05:04:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.31.161 - - [13/Dec/2015:05:04:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+107.173.176.148 - - [13/Dec/2015:05:11:56 +0100] "GET /apache-log/access.log HTTP/1.1" 200 97106 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Iron/29.0.1600.1 Chrome/29.0.1600.1 Safari/537.36" "-"
+188.191.175.166 - - [13/Dec/2015:05:13:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:05:13:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.105.140.247 - - [13/Dec/2015:05:15:12 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.105.140.247 - - [13/Dec/2015:05:15:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:05:24:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:05:24:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:05:32:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:05:32:49 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:05:35:41 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:05:35:41 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.0.189.113 - - [13/Dec/2015:05:46:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.0.189.113 - - [13/Dec/2015:05:46:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.78.52.106 - - [13/Dec/2015:05:46:56 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.78.52.106 - - [13/Dec/2015:05:46:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [13/Dec/2015:05:52:32 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [13/Dec/2015:05:52:33 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+207.46.13.57 - - [13/Dec/2015:05:52:43 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+85.26.232.27 - - [13/Dec/2015:05:52:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.26.232.27 - - [13/Dec/2015:05:52:49 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:05:56:04 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:05:56:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:06:00:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:06:00:48 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:06:06:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:06:06:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:06:19:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:06:19:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.72 - - [13/Dec/2015:06:20:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.72 - - [13/Dec/2015:06:20:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.135.72 - - [13/Dec/2015:06:22:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.153.135.72 - - [13/Dec/2015:06:22:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.200.76.18 - - [13/Dec/2015:06:23:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.200.76.18 - - [13/Dec/2015:06:23:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:06:23:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:06:23:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.45.44.138 - - [13/Dec/2015:06:24:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.45.44.138 - - [13/Dec/2015:06:24:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.146.123.167 - - [13/Dec/2015:06:32:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.146.123.167 - - [13/Dec/2015:06:32:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.25.206.81 - - [13/Dec/2015:06:32:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.25.206.81 - - [13/Dec/2015:06:32:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:06:34:59 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:06:34:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.19 - - [13/Dec/2015:06:35:51 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+95.24.218.160 - - [13/Dec/2015:06:38:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.24.218.160 - - [13/Dec/2015:06:38:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [13/Dec/2015:06:38:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+201.79.97.230 - - [13/Dec/2015:06:38:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:06:39:52 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:06:39:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:06:39:52 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+189.12.158.85 - - [13/Dec/2015:06:44:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+189.12.158.85 - - [13/Dec/2015:06:44:03 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.61.245.72 - - [13/Dec/2015:06:49:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.61.245.72 - - [13/Dec/2015:06:49:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.254.205 - - [13/Dec/2015:06:51:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.254.205 - - [13/Dec/2015:06:51:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.186.202.222 - - [13/Dec/2015:06:54:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+194.186.202.222 - - [13/Dec/2015:06:54:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.101.191 - - [13/Dec/2015:06:58:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.101.191 - - [13/Dec/2015:06:58:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:06:59:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:06:59:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:07:08:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:07:08:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:07:14:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.178.69 - - [13/Dec/2015:07:14:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:07:17:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:07:17:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.145.211 - - [13/Dec/2015:07:19:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.145.211 - - [13/Dec/2015:07:19:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.146.123.167 - - [13/Dec/2015:07:21:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.146.123.167 - - [13/Dec/2015:07:21:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+101.226.166.213 - - [13/Dec/2015:07:26:38 +0100] "GET /robots.txt HTTP/1.1" 200 304 "http://www.almhuette-raith.at/robots.txt" "Mozilla/5.0 (Windows NT 6.1) AppleWebKit/537.1 (KHTML, like Gecko) Chrome/21.0.1180.89 Safari/537.1; 360Spider(compatible; HaosouSpider; http://www.haosou.com/help/help_3_2.html)" "-"
+109.194.45.159 - - [13/Dec/2015:07:27:04 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.194.45.159 - - [13/Dec/2015:07:27:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [13/Dec/2015:07:34:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.90.110 - - [13/Dec/2015:07:34:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+67.17.36.130 - - [13/Dec/2015:07:35:21 +0100] "GET /apache-log/access.log HTTP/1.0" 200 112297 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+67.17.36.130 - - [13/Dec/2015:07:35:26 +0100] "GET /robots.txt HTTP/1.1" 200 304 "http://www.almhuette-raith.at/apache-log/access.log" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+187.58.71.107 - - [13/Dec/2015:07:37:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.58.71.107 - - [13/Dec/2015:07:37:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.126.5.50 - - [13/Dec/2015:07:37:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.126.5.50 - - [13/Dec/2015:07:37:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.141.88.217 - - [13/Dec/2015:07:40:41 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.141.88.217 - - [13/Dec/2015:07:40:41 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.127.71 - - [13/Dec/2015:07:44:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.127.71 - - [13/Dec/2015:07:44:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+111.14.40.153 - - [13/Dec/2015:07:46:43 +0100] "GET /apache-log/access.log HTTP/1.0" 200 114324 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+94.233.8.101 - - [13/Dec/2015:07:47:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.8.101 - - [13/Dec/2015:07:47:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:07:51:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:07:51:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:07:53:47 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:07:53:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:07:53:47 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+46.166.75.133 - - [13/Dec/2015:07:54:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.166.75.133 - - [13/Dec/2015:07:54:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.26.234.107 - - [13/Dec/2015:07:55:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.26.234.107 - - [13/Dec/2015:07:55:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:07:55:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:07:55:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:08:00:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:08:00:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.61.245.72 - - [13/Dec/2015:08:05:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.61.245.72 - - [13/Dec/2015:08:05:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.129.14.85 - - [13/Dec/2015:08:05:09 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.129.14.85 - - [13/Dec/2015:08:05:10 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:08:09:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.98.255.56 - - [13/Dec/2015:08:09:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.232.140.235 - - [13/Dec/2015:08:12:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.232.140.235 - - [13/Dec/2015:08:12:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.159.102.216 - - [13/Dec/2015:08:17:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.159.102.216 - - [13/Dec/2015:08:17:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.22 - - [13/Dec/2015:08:17:54 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+178.57.114.158 - - [13/Dec/2015:08:18:38 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.57.114.158 - - [13/Dec/2015:08:18:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.113.244.223 - - [13/Dec/2015:08:21:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.113.244.223 - - [13/Dec/2015:08:21:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+190.85.50.123 - - [13/Dec/2015:08:23:06 +0100] "GET /apache-log/access.log HTTP/1.1" 200 120422 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+190.85.50.123 - - [13/Dec/2015:08:23:09 +0100] "GET /robots.txt HTTP/1.1" 200 304 "http://www.almhuette-raith.at/apache-log/access.log" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+87.255.246.50 - - [13/Dec/2015:08:24:38 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:08:24:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.191.144.222 - - [13/Dec/2015:08:26:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.191.144.222 - - [13/Dec/2015:08:26:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:08:27:49 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:08:27:49 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.245.117.132 - - [13/Dec/2015:08:30:38 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.245.117.132 - - [13/Dec/2015:08:30:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:08:32:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:08:32:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [13/Dec/2015:08:33:44 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.167.43.54 - - [13/Dec/2015:08:33:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.199.113 - - [13/Dec/2015:08:37:43 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.199.113 - - [13/Dec/2015:08:37:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.28.162.22 - - [13/Dec/2015:08:40:37 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.28.162.22 - - [13/Dec/2015:08:40:37 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:08:40:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:08:40:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.237.175 - - [13/Dec/2015:08:40:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.78.237.175 - - [13/Dec/2015:08:40:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:08:43:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.70.106.8 - - [13/Dec/2015:08:43:10 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.117.38.167 - - [13/Dec/2015:08:43:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.117.38.167 - - [13/Dec/2015:08:43:33 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.210.113.241 - - [13/Dec/2015:08:44:03 +0100] "GET /apache-log/access.log HTTP/1.1" 200 125455 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.4.600 Chrome/30.0.1599.101 Safari/537.36" "-"
+95.47.158.158 - - [13/Dec/2015:08:49:30 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.47.158.158 - - [13/Dec/2015:08:49:30 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.112.238.242 - - [13/Dec/2015:08:50:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+187.112.238.242 - - [13/Dec/2015:08:50:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:08:51:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:08:51:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.34.161.183 - - [13/Dec/2015:08:51:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.34.161.183 - - [13/Dec/2015:08:51:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.208.30 - - [13/Dec/2015:08:54:41 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.0.208.30 - - [13/Dec/2015:08:54:41 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:08:57:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:08:57:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.151.33 - - [13/Dec/2015:08:59:51 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.233.151.33 - - [13/Dec/2015:08:59:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.132.225.75 - - [13/Dec/2015:09:00:07 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.132.225.75 - - [13/Dec/2015:09:00:07 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.163.98.48 - - [13/Dec/2015:09:01:40 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.163.98.48 - - [13/Dec/2015:09:01:40 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.255.253.120 - - [13/Dec/2015:09:01:48 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+130.193.50.25 - - [13/Dec/2015:09:01:49 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+5.255.253.120 - - [13/Dec/2015:09:01:53 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+130.193.50.25 - - [13/Dec/2015:09:01:53 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+130.193.50.25 - - [13/Dec/2015:09:02:03 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+5.255.253.120 - - [13/Dec/2015:09:02:03 +0100] "GET / HTTP/1.1" 200 10439 "-" "Mozilla/5.0 (compatible; YandexBot/3.0; +http://yandex.com/bots)" "-"
+91.227.190.44 - - [13/Dec/2015:09:03:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.190.44 - - [13/Dec/2015:09:03:33 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.22.237 - - [13/Dec/2015:09:04:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.22.237 - - [13/Dec/2015:09:04:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+88.200.214.77 - - [13/Dec/2015:09:05:56 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+88.200.214.77 - - [13/Dec/2015:09:05:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [13/Dec/2015:09:07:14 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.105.13.78 - - [13/Dec/2015:09:07:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.34.113.155 - - [13/Dec/2015:09:07:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.34.113.155 - - [13/Dec/2015:09:07:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+180.76.15.139 - - [13/Dec/2015:09:12:27 +0100] "GET /index.php?format=feed&type=rss HTTP/1.1" 200 3730 "-" "Mozilla/5.0 (compatible; Baiduspider/2.0; +http://www.baidu.com/search/spider.html)" "-"
+95.188.224.197 - - [13/Dec/2015:09:13:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.188.224.197 - - [13/Dec/2015:09:13:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.133.214.131 - - [13/Dec/2015:09:14:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.133.214.131 - - [13/Dec/2015:09:14:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.36.189.241 - - [13/Dec/2015:09:14:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.36.189.241 - - [13/Dec/2015:09:14:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.37.231.110 - - [13/Dec/2015:09:15:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.37.231.110 - - [13/Dec/2015:09:15:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+206.59.253.65 - - [13/Dec/2015:09:17:30 +0100] "GET /apache-log/access.log HTTP/1.1" 200 133584 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+37.19.35.108 - - [13/Dec/2015:09:20:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.19.35.108 - - [13/Dec/2015:09:20:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.234.14.220 - - [13/Dec/2015:09:21:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.234.14.220 - - [13/Dec/2015:09:21:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.32.68.77 - - [13/Dec/2015:09:25:43 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.32.68.77 - - [13/Dec/2015:09:25:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.125.155 - - [13/Dec/2015:09:27:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.125.155 - - [13/Dec/2015:09:27:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:09:31:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.191.175.166 - - [13/Dec/2015:09:31:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:09:31:07 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.204.13.65 - - [13/Dec/2015:09:31:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.87.79 - - [13/Dec/2015:09:31:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.87.79 - - [13/Dec/2015:09:31:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.174.39.161 - - [13/Dec/2015:09:43:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.174.39.161 - - [13/Dec/2015:09:43:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+103.53.16.6 - - [13/Dec/2015:09:49:34 +0100] "GET /apache-log/access.log HTTP/1.0" 200 136844 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+85.95.187.231 - - [13/Dec/2015:09:54:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.95.187.231 - - [13/Dec/2015:09:54:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.124.168.83 - - [13/Dec/2015:09:57:37 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.124.168.83 - - [13/Dec/2015:09:57:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.207.76 - - [13/Dec/2015:09:59:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.207.76 - - [13/Dec/2015:09:59:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:10:00:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:10:00:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:10:02:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.143.62 - - [13/Dec/2015:10:02:48 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.221.14.168 - - [13/Dec/2015:10:04:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.221.14.168 - - [13/Dec/2015:10:04:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.103.133 - - [13/Dec/2015:10:05:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.87.103.133 - - [13/Dec/2015:10:05:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.122.96.106 - - [13/Dec/2015:10:10:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.122.96.106 - - [13/Dec/2015:10:10:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.26.14 - - [13/Dec/2015:10:11:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.26.14 - - [13/Dec/2015:10:11:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.134.205.87 - - [13/Dec/2015:10:14:39 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.134.205.87 - - [13/Dec/2015:10:14:39 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+146.148.50.161 - - [13/Dec/2015:10:15:40 +0100] "GET /apache-log/access.log HTTP/1.1" 200 140856 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+2.93.22.237 - - [13/Dec/2015:10:16:14 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.22.237 - - [13/Dec/2015:10:16:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.125.155 - - [13/Dec/2015:10:16:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.125.155 - - [13/Dec/2015:10:16:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [13/Dec/2015:10:18:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [13/Dec/2015:10:18:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.210.201 - - [13/Dec/2015:10:23:06 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.210.201 - - [13/Dec/2015:10:23:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.219.208.1 - - [13/Dec/2015:10:23:58 +0100] "GET /apache-log/access.log HTTP/1.1" 200 142619 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+83.220.236.21 - - [13/Dec/2015:10:24:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.220.236.21 - - [13/Dec/2015:10:24:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.29.219 - - [13/Dec/2015:10:24:47 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.29.219 - - [13/Dec/2015:10:24:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.111.186.153 - - [13/Dec/2015:10:25:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.111.186.153 - - [13/Dec/2015:10:25:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:10:26:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.193.76.80 - - [13/Dec/2015:10:26:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.164.219 - - [13/Dec/2015:10:26:41 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.180.164.219 - - [13/Dec/2015:10:26:41 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+52.192.135.95 - - [13/Dec/2015:10:28:14 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/4.0 (compatible; MSIE8.0; Windows NT 6.0) .NET CLR 2.0.50727)" "-"
+91.227.29.79 - - [13/Dec/2015:10:28:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.29.79 - - [13/Dec/2015:10:28:20 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.29 - - [13/Dec/2015:10:33:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.29 - - [13/Dec/2015:10:33:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:10:33:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.255.246.50 - - [13/Dec/2015:10:33:11 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.172.112.180 - - [13/Dec/2015:10:33:37 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.172.112.180 - - [13/Dec/2015:10:33:37 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.88.222.215 - - [13/Dec/2015:10:35:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.88.222.215 - - [13/Dec/2015:10:35:48 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.26.14 - - [13/Dec/2015:10:36:47 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.26.14 - - [13/Dec/2015:10:36:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.229.140.120 - - [13/Dec/2015:10:37:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.229.140.120 - - [13/Dec/2015:10:37:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+192.162.212.213 - - [13/Dec/2015:10:39:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+192.162.212.213 - - [13/Dec/2015:10:39:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.133.214.131 - - [13/Dec/2015:10:41:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.133.214.131 - - [13/Dec/2015:10:41:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.7.156 - - [13/Dec/2015:10:43:32 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.7.156 - - [13/Dec/2015:10:43:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.157.235.169 - - [13/Dec/2015:10:49:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.157.235.169 - - [13/Dec/2015:10:49:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+104.154.90.76 - - [13/Dec/2015:10:49:21 +0100] "GET /apache-log/access.log HTTP/1.0" 200 149042 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+93.76.246.140 - - [13/Dec/2015:10:51:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:10:51:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.136.213.198 - - [13/Dec/2015:10:51:26 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.136.213.198 - - [13/Dec/2015:10:51:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.19.35.192 - - [13/Dec/2015:10:51:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.19.35.192 - - [13/Dec/2015:10:51:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.36.198 - - [13/Dec/2015:10:53:13 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.36.198 - - [13/Dec/2015:10:53:13 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.83.171.234 - - [13/Dec/2015:10:54:11 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.83.171.234 - - [13/Dec/2015:10:54:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.225.200.175 - - [13/Dec/2015:10:58:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.225.200.175 - - [13/Dec/2015:10:58:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [13/Dec/2015:10:58:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.215.38.205 - - [13/Dec/2015:10:58:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.180.50.222 - - [13/Dec/2015:10:59:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.180.50.222 - - [13/Dec/2015:10:59:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+66.249.66.19 - - [13/Dec/2015:11:02:42 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+89.28.162.22 - - [13/Dec/2015:11:05:07 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.28.162.22 - - [13/Dec/2015:11:05:07 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:11:05:56 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:11:05:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.139.193.228 - - [13/Dec/2015:11:06:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.139.193.228 - - [13/Dec/2015:11:06:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.34.203 - - [13/Dec/2015:11:08:30 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.34.203 - - [13/Dec/2015:11:08:30 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.181.253.16 - - [13/Dec/2015:11:10:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+31.181.253.16 - - [13/Dec/2015:11:10:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.3.32.17 - - [13/Dec/2015:11:10:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.3.32.17 - - [13/Dec/2015:11:10:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:11:14:23 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:11:14:23 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+208.123.223.152 - - [13/Dec/2015:11:14:50 +0100] "GET /apache-log/access.log HTTP/1.1" 200 155085 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+46.44.5.179 - - [13/Dec/2015:11:16:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.44.5.179 - - [13/Dec/2015:11:16:30 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.173.94.97 - - [13/Dec/2015:11:19:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.173.94.97 - - [13/Dec/2015:11:19:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.68.33 - - [13/Dec/2015:11:19:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.76.68.33 - - [13/Dec/2015:11:19:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.96.253.187 - - [13/Dec/2015:11:20:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.96.253.187 - - [13/Dec/2015:11:20:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+212.95.7.131 - - [13/Dec/2015:11:28:41 +0100] "GET /templates/_system/css/general.css HTTP/1.1" 404 239 "http://www.almhuette-raith.at/" "Mozilla/5.0 (iPhone; CPU iPhone OS 9_1 like Mac OS X) AppleWebKit/601.1.46 (KHTML, like Gecko) Version/9.0 Mobile/13B143 Safari/601.1" "-"
+91.211.128.248 - - [13/Dec/2015:11:29:27 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.211.128.248 - - [13/Dec/2015:11:29:27 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.78.35.50 - - [13/Dec/2015:11:30:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.11 - - [13/Dec/2015:11:32:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.11 - - [13/Dec/2015:11:32:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.68.202.81 - - [13/Dec/2015:11:34:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.68.202.81 - - [13/Dec/2015:11:34:56 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.70.135.96 - - [13/Dec/2015:11:36:20 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.70.135.96 - - [13/Dec/2015:11:36:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.34.113.155 - - [13/Dec/2015:11:37:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.34.113.155 - - [13/Dec/2015:11:37:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+67.17.36.173 - - [13/Dec/2015:11:40:01 +0100] "GET /apache-log/access.log HTTP/1.0" 200 159154 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+177.203.52.133 - - [13/Dec/2015:11:41:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.203.52.133 - - [13/Dec/2015:11:41:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.125.23 - - [13/Dec/2015:11:43:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.125.23 - - [13/Dec/2015:11:43:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.214.173.147 - - [13/Dec/2015:11:43:42 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.214.173.147 - - [13/Dec/2015:11:43:42 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.100.31.117 - - [13/Dec/2015:11:44:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+92.100.31.117 - - [13/Dec/2015:11:44:02 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:11:45:04 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:11:45:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:11:45:04 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+109.197.194.109 - - [13/Dec/2015:11:48:57 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.197.194.109 - - [13/Dec/2015:11:48:57 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:11:50:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:11:50:24 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.84.175.69 - - [13/Dec/2015:11:50:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.84.175.69 - - [13/Dec/2015:11:50:26 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.49.68.156 - - [13/Dec/2015:11:52:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.49.68.156 - - [13/Dec/2015:11:52:10 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.165.200.217 - - [13/Dec/2015:11:55:01 +0100] "HEAD / HTTP/1.1" 200 0 "-" "Mozilla/4.0 (compatible; MSIE 5.01; Windows NT 5.0)" "-"
+81.20.205.237 - - [13/Dec/2015:11:55:25 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.20.205.237 - - [13/Dec/2015:11:55:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.124.168.83 - - [13/Dec/2015:11:55:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.124.168.83 - - [13/Dec/2015:11:55:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.128.231 - - [13/Dec/2015:11:56:18 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.128.231 - - [13/Dec/2015:11:56:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.194.74.20 - - [13/Dec/2015:11:56:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.194.74.20 - - [13/Dec/2015:11:56:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.206.176.201 - - [13/Dec/2015:11:58:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.206.176.201 - - [13/Dec/2015:11:58:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.17.60.192 - - [13/Dec/2015:12:01:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.17.60.192 - - [13/Dec/2015:12:01:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.7.23 - - [13/Dec/2015:12:01:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.7.23 - - [13/Dec/2015:12:01:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.185.22.247 - - [13/Dec/2015:12:01:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.185.22.247 - - [13/Dec/2015:12:01:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+40.77.167.66 - - [13/Dec/2015:12:05:08 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+40.77.167.66 - - [13/Dec/2015:12:05:25 +0100] "GET /apache-log/error.log.44.gz HTTP/1.1" 404 232 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+185.103.26.14 - - [13/Dec/2015:12:06:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.26.14 - - [13/Dec/2015:12:06:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.187.32.254 - - [13/Dec/2015:12:07:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.187.32.254 - - [13/Dec/2015:12:07:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.169.248.247 - - [13/Dec/2015:12:10:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.169.248.247 - - [13/Dec/2015:12:10:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.27.89 - - [13/Dec/2015:12:12:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.103.27.89 - - [13/Dec/2015:12:12:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.29 - - [13/Dec/2015:12:13:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.69.0.29 - - [13/Dec/2015:12:13:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.167.78.166 - - [13/Dec/2015:12:16:02 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.167.78.166 - - [13/Dec/2015:12:16:03 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.49.68.156 - - [13/Dec/2015:12:18:04 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.49.68.156 - - [13/Dec/2015:12:18:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [13/Dec/2015:12:19:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.72.185.236 - - [13/Dec/2015:12:19:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.81.240.26 - - [13/Dec/2015:12:19:37 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+78.81.240.26 - - [13/Dec/2015:12:19:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+157.55.39.7 - - [13/Dec/2015:12:21:07 +0100] "GET /robots.txt HTTP/1.1" 200 304 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+157.55.39.8 - - [13/Dec/2015:12:21:13 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; bingbot/2.0; +http://www.bing.com/bingbot.htm)" "-"
+81.20.102.154 - - [13/Dec/2015:12:21:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+81.20.102.154 - - [13/Dec/2015:12:21:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:12:26:43 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:12:26:43 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:12:27:48 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.44.113.180 - - [13/Dec/2015:12:27:48 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.165.15.204 - - [13/Dec/2015:12:28:58 +0100] "GET /index.php?option=com_phocagallery&view=category&id=1&Itemid=53&limitstart=20 HTTP/1.1" 200 15436 "-" "Mozilla/5.0 (compatible; AhrefsBot/5.0; +http://ahrefs.com/robot/)" "-"
+128.73.20.111 - - [13/Dec/2015:12:30:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.73.20.111 - - [13/Dec/2015:12:30:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.203.170.128 - - [13/Dec/2015:12:31:34 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.203.170.128 - - [13/Dec/2015:12:31:34 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+23.95.241.115 - - [13/Dec/2015:12:32:27 +0100] "GET /apache-log/access.log HTTP/1.0" 200 172495 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+66.249.66.19 - - [13/Dec/2015:12:33:58 +0100] "GET / HTTP/1.1" 200 10479 "-" "Mozilla/5.0 (compatible; Googlebot/2.1; +http://www.google.com/bot.html)" "-"
+185.3.32.17 - - [13/Dec/2015:12:35:09 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.3.32.17 - - [13/Dec/2015:12:35:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.11 - - [13/Dec/2015:12:38:59 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+80.66.158.11 - - [13/Dec/2015:12:39:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:12:42:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.76.246.140 - - [13/Dec/2015:12:42:45 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+82.193.155.224 - - [13/Dec/2015:12:43:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+82.193.155.224 - - [13/Dec/2015:12:43:29 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.169.248.247 - - [13/Dec/2015:12:43:46 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.169.248.247 - - [13/Dec/2015:12:43:46 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.71.187.170 - - [13/Dec/2015:12:44:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.71.187.170 - - [13/Dec/2015:12:44:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.125.23 - - [13/Dec/2015:12:46:06 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.125.23 - - [13/Dec/2015:12:46:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.129.207.169 - - [13/Dec/2015:12:47:01 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.129.207.169 - - [13/Dec/2015:12:47:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.3.32.17 - - [13/Dec/2015:12:47:21 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+185.3.32.17 - - [13/Dec/2015:12:47:21 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.31.161 - - [13/Dec/2015:12:48:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.140.31.161 - - [13/Dec/2015:12:48:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:12:52:45 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:12:52:46 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.120.59.191 - - [13/Dec/2015:12:54:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+188.120.59.191 - - [13/Dec/2015:12:54:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.141.91.118 - - [13/Dec/2015:12:55:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.141.91.118 - - [13/Dec/2015:12:55:01 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.146.101.241 - - [13/Dec/2015:12:55:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.146.101.241 - - [13/Dec/2015:12:55:06 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:12:55:58 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:12:56:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:12:56:00 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+87.117.14.0 - - [13/Dec/2015:12:59:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.117.14.0 - - [13/Dec/2015:12:59:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.29.79 - - [13/Dec/2015:13:00:40 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.227.29.79 - - [13/Dec/2015:13:00:40 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+206.59.253.65 - - [13/Dec/2015:13:01:25 +0100] "GET /apache-log/access.log HTTP/1.1" 200 179700 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+109.184.11.34 - - [13/Dec/2015:13:07:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.184.11.34 - - [13/Dec/2015:13:07:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.166.75.133 - - [13/Dec/2015:13:11:14 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.166.75.133 - - [13/Dec/2015:13:11:14 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.224.215 - - [13/Dec/2015:13:13:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.29.224.215 - - [13/Dec/2015:13:13:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.106.125 - - [13/Dec/2015:13:16:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.53.106.125 - - [13/Dec/2015:13:16:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.232.118.5 - - [13/Dec/2015:13:18:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.232.118.5 - - [13/Dec/2015:13:18:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+90.154.66.233 - - [13/Dec/2015:13:19:08 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+90.154.66.233 - - [13/Dec/2015:13:19:08 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.210.113.241 - - [13/Dec/2015:13:19:42 +0100] "GET /apache-log/access.log HTTP/1.1" 200 182212 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.4.600 Chrome/30.0.1599.101 Safari/537.36" "-"
+128.72.82.254 - - [13/Dec/2015:13:20:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+128.72.82.254 - - [13/Dec/2015:13:20:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.113.244.223 - - [13/Dec/2015:13:26:59 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.113.244.223 - - [13/Dec/2015:13:26:59 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [13/Dec/2015:13:27:33 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.141.121.10 - - [13/Dec/2015:13:27:33 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [13/Dec/2015:13:30:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.140.24.131 - - [13/Dec/2015:13:30:35 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.214.240.8 - - [13/Dec/2015:13:31:44 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.214.240.8 - - [13/Dec/2015:13:31:44 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.139.27.245 - - [13/Dec/2015:13:32:24 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.139.27.245 - - [13/Dec/2015:13:32:25 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.36.198 - - [13/Dec/2015:13:33:03 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.138.36.198 - - [13/Dec/2015:13:33:04 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.132.190 - - [13/Dec/2015:13:33:10 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+213.24.132.190 - - [13/Dec/2015:13:33:10 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.211.128.248 - - [13/Dec/2015:13:36:00 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.211.128.248 - - [13/Dec/2015:13:36:00 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:13:37:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+46.211.46.18 - - [13/Dec/2015:13:37:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+204.44.90.14 - - [13/Dec/2015:13:39:16 +0100] "GET /apache-log/access.log HTTP/1.0" 200 186228 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+204.44.90.14 - - [13/Dec/2015:13:39:18 +0100] "GET /robots.txt HTTP/1.1" 200 304 "http://www.almhuette-raith.at/apache-log/access.log" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+94.19.17.65 - - [13/Dec/2015:13:41:09 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+94.19.17.65 - - [13/Dec/2015:13:41:09 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.32.68.29 - - [13/Dec/2015:13:42:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.32.68.29 - - [13/Dec/2015:13:42:18 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.173.94.97 - - [13/Dec/2015:13:44:47 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+85.173.94.97 - - [13/Dec/2015:13:44:47 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.201.52.125 - - [13/Dec/2015:13:49:29 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+177.201.52.125 - - [13/Dec/2015:13:49:30 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:13:50:12 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:13:50:12 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.122.96.106 - - [13/Dec/2015:14:00:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+62.122.96.106 - - [13/Dec/2015:14:00:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.218.92.2 - - [13/Dec/2015:14:01:05 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.218.92.2 - - [13/Dec/2015:14:01:05 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.219.156.45 - - [13/Dec/2015:14:06:38 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.219.156.45 - - [13/Dec/2015:14:06:38 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.194.74.20 - - [13/Dec/2015:14:08:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+176.194.74.20 - - [13/Dec/2015:14:08:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [13/Dec/2015:14:09:28 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+130.255.13.57 - - [13/Dec/2015:14:09:28 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.117.38.218 - - [13/Dec/2015:14:11:35 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+87.117.38.218 - - [13/Dec/2015:14:11:36 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:14:13:53 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.166.200.19 - - [13/Dec/2015:14:13:53 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:14:14:55 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.167.113.100 - - [13/Dec/2015:14:14:55 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.244.21.14 - - [13/Dec/2015:14:17:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+77.244.21.14 - - [13/Dec/2015:14:17:51 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.86.149.141 - - [13/Dec/2015:14:19:18 +0100] "GET /apache-log/access.log HTTP/1.1" 200 191995 "http://www.almhuette-raith.at/" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Maxthon/4.4.3.4000 Chrome/30.0.1599.101 Safari/537.36" "-"
+109.184.11.34 - - [13/Dec/2015:14:23:17 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.184.11.34 - - [13/Dec/2015:14:23:17 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.22.237 - - [13/Dec/2015:14:23:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+2.93.22.237 - - [13/Dec/2015:14:23:52 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:14:25:58 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+109.106.142.176 - - [13/Dec/2015:14:25:58 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+195.8.51.14 - - [13/Dec/2015:14:28:15 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+195.8.51.14 - - [13/Dec/2015:14:28:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.214.187.254 - - [13/Dec/2015:14:29:19 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+91.214.187.254 - - [13/Dec/2015:14:29:19 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.227.6.83 - - [13/Dec/2015:14:29:50 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+5.227.6.83 - - [13/Dec/2015:14:29:50 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+37.1.206.196 - - [13/Dec/2015:14:30:30 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:14:30:31 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+37.1.206.196 - - [13/Dec/2015:14:30:31 +0100] "GET /administrator/index.php HTTP/1.1" 200 4263 "http://almhuette-raith.at/administrator/index.php" "Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/31.0.1650.57 Safari/537.36" "-"
+89.145.141.195 - - [13/Dec/2015:14:33:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.145.141.195 - - [13/Dec/2015:14:33:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.32.85.147 - - [13/Dec/2015:14:33:22 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+95.32.85.147 - - [13/Dec/2015:14:33:22 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.220.236.21 - - [13/Dec/2015:14:37:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+83.220.236.21 - - [13/Dec/2015:14:37:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [13/Dec/2015:14:37:16 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+89.178.243.193 - - [13/Dec/2015:14:37:16 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.29.219 - - [13/Dec/2015:14:40:31 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+178.35.29.219 - - [13/Dec/2015:14:40:32 +0100] "POST /administrator/index.php HTTP/1.1" 200 4494 "http://almhuette-raith.at/administrator/" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"
+93.185.22.218 - - [13/Dec/2015:14:40:52 +0100] "GET /administrator/ HTTP/1.1" 200 4263 "-" "Mozilla/5.0 (Windows NT 6.0; rv:34.0) Gecko/20100101 Firefox/34.0" "-"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org