mirror of https://github.com/apache/druid.git
Filters: Add filter.toFilter method, use that instead of the instanceof chain in Filters.
I believe that the instanceof chain in Filters exists because in the past, Filter and DimFilter were in different packages (DimFilter was in druid-client and Filter was in druid-processing). And since druid-client didn't depend on druid-processing, DimFilter couldn't have a toFilter method. But now it can.
This commit is contained in:
parent
a5d5529749
commit
713062053c
|
@ -60,7 +60,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
|||
public Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
|
||||
{
|
||||
final ValueMatcherFactory valueMatcherFactory = new FilteredAggregatorValueMatcherFactory(columnSelectorFactory);
|
||||
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(valueMatcherFactory);
|
||||
final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(valueMatcherFactory);
|
||||
return new FilteredAggregator(
|
||||
valueMatcher,
|
||||
delegate.factorize(columnSelectorFactory)
|
||||
|
@ -71,7 +71,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
|||
public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
|
||||
{
|
||||
final ValueMatcherFactory valueMatcherFactory = new FilteredAggregatorValueMatcherFactory(columnSelectorFactory);
|
||||
final ValueMatcher valueMatcher = Filters.convertDimensionFilters(filter).makeMatcher(valueMatcherFactory);
|
||||
final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(valueMatcherFactory);
|
||||
return new FilteredBufferAggregator(
|
||||
valueMatcher,
|
||||
delegate.factorizeBuffered(columnSelectorFactory)
|
||||
|
|
|
@ -26,6 +26,8 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.segment.filter.AndFilter;
|
||||
import io.druid.segment.filter.Filters;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -73,6 +75,12 @@ public class AndDimFilter implements DimFilter
|
|||
})).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new AndFilter(Filters.toFilters(fields));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.BoundFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -133,6 +134,12 @@ public class BoundDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new BoundFilter(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -48,4 +48,12 @@ public interface DimFilter
|
|||
* returning the same filter can be a straightforward default implementation.
|
||||
*/
|
||||
public DimFilter optimize();
|
||||
|
||||
/**
|
||||
* Returns a Filter that implements this DimFilter. This does not generally involve optimizing the DimFilter,
|
||||
* so it does make sense to optimize first and then call toFilter on the resulting DimFilter.
|
||||
*
|
||||
* @return a Filter that implements this DimFilter, or null if this DimFilter is a no-op.
|
||||
*/
|
||||
public Filter toFilter();
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import com.metamx.common.StringUtils;
|
|||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.lookup.LookupExtractionFn;
|
||||
import io.druid.query.lookup.LookupExtractor;
|
||||
import io.druid.segment.filter.ExtractionFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.List;
|
||||
|
@ -116,6 +117,12 @@ public class ExtractionDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new ExtractionFilter(dimension, value, extractionFn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -21,8 +21,13 @@ package io.druid.query.filter;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.OrFilter;
|
||||
import io.druid.segment.filter.SelectorFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
|
@ -70,7 +75,7 @@ public class InDimFilter implements DimFilter
|
|||
.put(DimFilterCacheHelper.IN_CACHE_ID)
|
||||
.put(dimensionBytes)
|
||||
.put(DimFilterCacheHelper.STRING_SEPARATOR);
|
||||
for (byte [] bytes: valuesBytes) {
|
||||
for (byte[] bytes : valuesBytes) {
|
||||
filterCacheKey.put(bytes)
|
||||
.put((byte) 0xFF);
|
||||
}
|
||||
|
@ -83,6 +88,26 @@ public class InDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
final List<Filter> selectorFilters = ImmutableList.copyOf(
|
||||
Iterables.transform(
|
||||
values,
|
||||
new Function<String, Filter>()
|
||||
{
|
||||
@Override
|
||||
public Filter apply(String input)
|
||||
{
|
||||
return new SelectorFilter(dimension, input);
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
|
||||
return new OrFilter(selectorFilters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.JavaScriptFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -75,6 +76,12 @@ public class JavaScriptDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new JavaScriptFilter(dimension, function);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -36,4 +36,10 @@ public class NoopDimFilter implements DimFilter
|
|||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.segment.filter.NotFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -61,6 +62,12 @@ public class NotDimFilter implements DimFilter
|
|||
return Druids.newNotDimFilterBuilder().field(this.getField().optimize()).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new NotFilter(field.toFilter());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -26,6 +26,8 @@ import com.google.common.base.Joiner;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.segment.filter.Filters;
|
||||
import io.druid.segment.filter.OrFilter;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -73,6 +75,12 @@ public class OrDimFilter implements DimFilter
|
|||
})).build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new OrFilter(Filters.toFilters(fields));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.RegexFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -77,6 +78,12 @@ public class RegexDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new RegexFilter(dimension, pattern);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.query.search.search.SearchQuerySpec;
|
||||
import io.druid.segment.filter.SearchQueryFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -77,6 +78,12 @@ public class SearchQueryDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new SearchQueryFilter(dimension, query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.SelectorFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -65,6 +66,12 @@ public class SelectorDimFilter implements DimFilter
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new SelectorFilter(dimension, value);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getDimension()
|
||||
{
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.metamx.collections.spatial.search.Bound;
|
||||
import com.metamx.common.StringUtils;
|
||||
import io.druid.segment.filter.SpatialFilter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
|
@ -78,6 +79,12 @@ public class SpatialDimFilter implements DimFilter
|
|||
return bound;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new SpatialFilter(dimension, bound);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
|
|
|
@ -95,7 +95,7 @@ public class GroupByQueryEngine
|
|||
}
|
||||
|
||||
final Sequence<Cursor> cursors = storageAdapter.makeCursors(
|
||||
Filters.convertDimensionFilters(query.getDimFilter()),
|
||||
Filters.toFilter(query.getDimFilter()),
|
||||
intervals.get(0),
|
||||
query.getGranularity(),
|
||||
false
|
||||
|
|
|
@ -85,7 +85,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
}
|
||||
|
||||
final SearchQuery query = (SearchQuery) input;
|
||||
final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter());
|
||||
final Filter filter = Filters.toFilter(query.getDimensionsFilter());
|
||||
final List<DimensionSpec> dimensions = query.getDimensions();
|
||||
final SearchQuerySpec searchQuerySpec = query.getQuery();
|
||||
final int limit = query.getLimit();
|
||||
|
|
|
@ -81,7 +81,7 @@ public class SelectQueryEngine
|
|||
return QueryRunnerHelper.makeCursorBasedQuery(
|
||||
adapter,
|
||||
query.getQuerySegmentSpec().getIntervals(),
|
||||
Filters.convertDimensionFilters(query.getDimensionsFilter()),
|
||||
Filters.toFilter(query.getDimensionsFilter()),
|
||||
query.isDescending(),
|
||||
query.getGranularity(),
|
||||
new Function<Cursor, Result<SelectResultValue>>()
|
||||
|
|
|
@ -47,7 +47,7 @@ public class TimeseriesQueryEngine
|
|||
return QueryRunnerHelper.makeCursorBasedQuery(
|
||||
adapter,
|
||||
query.getQuerySegmentSpec().getIntervals(),
|
||||
Filters.convertDimensionFilters(query.getDimensionsFilter()),
|
||||
Filters.toFilter(query.getDimensionsFilter()),
|
||||
query.isDescending(),
|
||||
query.getGranularity(),
|
||||
new Function<Cursor, Result<TimeseriesResultValue>>()
|
||||
|
|
|
@ -64,7 +64,7 @@ public class TopNQueryEngine
|
|||
}
|
||||
|
||||
final List<Interval> queryIntervals = query.getQuerySegmentSpec().getIntervals();
|
||||
final Filter filter = Filters.convertDimensionFilters(query.getDimensionsFilter());
|
||||
final Filter filter = Filters.toFilter(query.getDimensionsFilter());
|
||||
final QueryGranularity granularity = query.getGranularity();
|
||||
final Function<Cursor, Result<TopNResultValue>> mapFn = getMapFn(query, adapter);
|
||||
|
||||
|
|
|
@ -20,103 +20,51 @@
|
|||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.query.filter.AndDimFilter;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.common.guava.FunctionalIterable;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.ExtractionDimFilter;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.InDimFilter;
|
||||
import io.druid.query.filter.JavaScriptDimFilter;
|
||||
import io.druid.query.filter.NotDimFilter;
|
||||
import io.druid.query.filter.OrDimFilter;
|
||||
import io.druid.query.filter.RegexDimFilter;
|
||||
import io.druid.query.filter.SearchQueryDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.query.filter.SpatialDimFilter;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class Filters
|
||||
{
|
||||
public static List<Filter> convertDimensionFilters(List<DimFilter> filters)
|
||||
/**
|
||||
* Convert a list of DimFilters to a list of Filters.
|
||||
*
|
||||
* @param dimFilters list of DimFilters, should all be non-null
|
||||
*
|
||||
* @return list of Filters
|
||||
*/
|
||||
public static List<Filter> toFilters(List<DimFilter> dimFilters)
|
||||
{
|
||||
return Lists.transform(
|
||||
filters,
|
||||
new Function<DimFilter, Filter>()
|
||||
{
|
||||
@Override
|
||||
public Filter apply(@Nullable DimFilter input)
|
||||
{
|
||||
return convertDimensionFilters(input);
|
||||
}
|
||||
}
|
||||
return ImmutableList.copyOf(
|
||||
FunctionalIterable
|
||||
.create(dimFilters)
|
||||
.transform(
|
||||
new Function<DimFilter, Filter>()
|
||||
{
|
||||
@Override
|
||||
public Filter apply(DimFilter input)
|
||||
{
|
||||
return input.toFilter();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
public static Filter convertDimensionFilters(DimFilter dimFilter)
|
||||
/**
|
||||
* Convert a DimFilter to a Filter.
|
||||
*
|
||||
* @param dimFilter dimFilter
|
||||
*
|
||||
* @return converted filter, or null if input was null
|
||||
*/
|
||||
public static Filter toFilter(DimFilter dimFilter)
|
||||
{
|
||||
if (dimFilter == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
Filter filter = null;
|
||||
if (dimFilter instanceof AndDimFilter) {
|
||||
filter = new AndFilter(convertDimensionFilters(((AndDimFilter) dimFilter).getFields()));
|
||||
} else if (dimFilter instanceof OrDimFilter) {
|
||||
filter = new OrFilter(convertDimensionFilters(((OrDimFilter) dimFilter).getFields()));
|
||||
} else if (dimFilter instanceof NotDimFilter) {
|
||||
filter = new NotFilter(convertDimensionFilters(((NotDimFilter) dimFilter).getField()));
|
||||
} else if (dimFilter instanceof SelectorDimFilter) {
|
||||
final SelectorDimFilter selectorDimFilter = (SelectorDimFilter) dimFilter;
|
||||
|
||||
filter = new SelectorFilter(selectorDimFilter.getDimension(), selectorDimFilter.getValue());
|
||||
} else if (dimFilter instanceof ExtractionDimFilter) {
|
||||
final ExtractionDimFilter extractionDimFilter = (ExtractionDimFilter) dimFilter;
|
||||
|
||||
filter = new ExtractionFilter(
|
||||
extractionDimFilter.getDimension(),
|
||||
extractionDimFilter.getValue(),
|
||||
extractionDimFilter.getExtractionFn()
|
||||
);
|
||||
} else if (dimFilter instanceof RegexDimFilter) {
|
||||
final RegexDimFilter regexDimFilter = (RegexDimFilter) dimFilter;
|
||||
|
||||
filter = new RegexFilter(regexDimFilter.getDimension(), regexDimFilter.getPattern());
|
||||
} else if (dimFilter instanceof SearchQueryDimFilter) {
|
||||
final SearchQueryDimFilter searchQueryFilter = (SearchQueryDimFilter) dimFilter;
|
||||
|
||||
filter = new SearchQueryFilter(searchQueryFilter.getDimension(), searchQueryFilter.getQuery());
|
||||
} else if (dimFilter instanceof JavaScriptDimFilter) {
|
||||
final JavaScriptDimFilter javaScriptDimFilter = (JavaScriptDimFilter) dimFilter;
|
||||
|
||||
filter = new JavaScriptFilter(javaScriptDimFilter.getDimension(), javaScriptDimFilter.getFunction());
|
||||
} else if (dimFilter instanceof SpatialDimFilter) {
|
||||
final SpatialDimFilter spatialDimFilter = (SpatialDimFilter) dimFilter;
|
||||
|
||||
filter = new SpatialFilter(spatialDimFilter.getDimension(), spatialDimFilter.getBound());
|
||||
} else if (dimFilter instanceof InDimFilter) {
|
||||
final InDimFilter inDimFilter = (InDimFilter) dimFilter;
|
||||
final List<Filter> listFilters = Lists.transform(
|
||||
inDimFilter.getValues(), new Function<String, Filter>()
|
||||
{
|
||||
@Nullable
|
||||
@Override
|
||||
public Filter apply(@Nullable String input)
|
||||
{
|
||||
return new SelectorFilter(inDimFilter.getDimension(), input);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
filter = new OrFilter(listFilters);
|
||||
} else if (dimFilter instanceof BoundDimFilter) {
|
||||
filter = new BoundFilter((BoundDimFilter) dimFilter);
|
||||
}
|
||||
|
||||
return filter;
|
||||
return dimFilter == null ? null : dimFilter.toFilter();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -172,7 +172,7 @@ public class ExtractionDimFilterTest
|
|||
public void testOr()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
1, Filters.convertDimensionFilters(
|
||||
1, Filters.toFilter(
|
||||
DimFilters.or(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
@ -186,7 +186,7 @@ public class ExtractionDimFilterTest
|
|||
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
Filters.convertDimensionFilters(
|
||||
Filters.toFilter(
|
||||
DimFilters.or(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
@ -209,7 +209,7 @@ public class ExtractionDimFilterTest
|
|||
public void testAnd()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
1, Filters.convertDimensionFilters(
|
||||
1, Filters.toFilter(
|
||||
DimFilters.or(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
@ -223,7 +223,7 @@ public class ExtractionDimFilterTest
|
|||
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
Filters.convertDimensionFilters(
|
||||
Filters.toFilter(
|
||||
DimFilters.and(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
@ -247,7 +247,7 @@ public class ExtractionDimFilterTest
|
|||
{
|
||||
|
||||
Assert.assertEquals(
|
||||
1, Filters.convertDimensionFilters(
|
||||
1, Filters.toFilter(
|
||||
DimFilters.or(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
@ -261,7 +261,7 @@ public class ExtractionDimFilterTest
|
|||
|
||||
Assert.assertEquals(
|
||||
1,
|
||||
Filters.convertDimensionFilters(
|
||||
Filters.toFilter(
|
||||
DimFilters.not(
|
||||
new ExtractionDimFilter(
|
||||
"foo",
|
||||
|
|
|
@ -73,7 +73,7 @@ public class IngestSegmentFirehose implements Firehose
|
|||
return Sequences.concat(
|
||||
Sequences.map(
|
||||
adapter.getAdapter().makeCursors(
|
||||
Filters.convertDimensionFilters(dimFilter),
|
||||
Filters.toFilter(dimFilter),
|
||||
adapter.getInterval(),
|
||||
granularity,
|
||||
false
|
||||
|
|
Loading…
Reference in New Issue