You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2020/05/15 03:49:15 UTC

[GitHub] [incubator-hudi] nsivabalan commented on a change in pull request #1433: [HUDI-728]: Implement custom key generator

nsivabalan commented on a change in pull request #1433:
URL: https://github.com/apache/incubator-hudi/pull/1433#discussion_r425547719



##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
##########
@@ -66,6 +68,14 @@ public HoodieKey getKey(GenericRecord record) {
       partitionPath = partitionPathField + "=" + partitionPath;
     }
 
-    return new HoodieKey(recordKey, partitionPath);
+    return partitionPath;
+  }
+
+  public String getRecordKey(GenericRecord record) {

Review comment:
       did you think if we need to make this an abstract method in KeyGenerator? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.keygen;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.config.TypedProperties;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
+ * Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
+ * "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
+ *
+ * properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
+ *
+ * The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
+ *
+ * Few points to consider:
+ * 1. If you want to customise some partition path field on a timestamp basis, you can use field1:timestampBased

Review comment:
       minor typo. "customize"

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
##########
@@ -49,21 +50,44 @@
 
   public ComplexKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.asList(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","))
-            .stream().map(String::trim).collect(Collectors.toList());
-    this.partitionPathFields =
-        Arrays.asList(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","))
-                .stream().map(String::trim).collect(Collectors.toList());
+    DataSourceUtils.checkRequiredProperties(props, Arrays.asList(
+        DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(),
+        DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY())
+    );
+    this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(","))
+        .map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields = Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(","))
+        .map(String::trim).collect(Collectors.toList());
     this.hiveStylePartitioning = props.getBoolean(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY(),
         Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL()));
   }
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
-    if (recordKeyFields == null || partitionPathFields == null) {
-      throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
+    String recordKey = getRecordKey(record);
+    StringBuilder partitionPath = new StringBuilder();
+    for (String partitionPathField : partitionPathFields) {
+      partitionPath.append(getPartitionPath(record, partitionPathField));
+      partitionPath.append(DEFAULT_PARTITION_PATH_SEPARATOR);
+    }
+    partitionPath.deleteCharAt(partitionPath.length() - 1);
+
+    return new HoodieKey(recordKey, partitionPath.toString());
+  }
+
+  public String getPartitionPath(GenericRecord record, String partitionPathField) {

Review comment:
       does this needs to be public? why not protected or package private? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
##########
@@ -66,6 +68,14 @@ public HoodieKey getKey(GenericRecord record) {
       partitionPath = partitionPathField + "=" + partitionPath;
     }
 
-    return new HoodieKey(recordKey, partitionPath);
+    return partitionPath;
+  }
+
+  public String getRecordKey(GenericRecord record) {

Review comment:
       same here. does this need to be public? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
##########
@@ -88,6 +86,13 @@ public TimestampBasedKeyGenerator(TypedProperties config) {
 
   @Override
   public HoodieKey getKey(GenericRecord record) {
+    String recordKey = getRecordKey(record);

Review comment:
       a naive question. May I know why some key generators of them are extending from SimpleKeyGenerator and some are extending KeyGenerators. Even those extending from SimpleKeyGen, I don't see any obvious need to extend from SimpleKeyGenerator. Can you throw some light. 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.keygen;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.config.TypedProperties;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
+ * Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
+ * "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
+ *
+ * properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
+ *
+ * The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
+ *
+ * Few points to consider:
+ * 1. If you want to customise some partition path field on a timestamp basis, you can use field1:timestampBased
+ * 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
+ * 3. If you want your table to be non partitioned, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List<String> recordKeyFields;
+  protected final List<String> partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+    SIMPLE, TIMESTAMP
+  }
+
+  public CustomKeyGenerator(TypedProperties props) {
+    super(props);
+    this.properties = props;
+    this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields =
+      Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+  }
+
+  @Override
+  public HoodieKey getKey(GenericRecord record) {
+    //call function to get the record key
+    String recordKey = getRecordKey(record);
+    //call function to get the partition key based on the type for that partition path field
+    String partitionPath = getPartitionPath(record);
+    return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {
+    if (partitionPathFields == null) {
+      throw new HoodieKeyException("Unable to find field names for partition path in cfg");
+    }
+
+    String partitionPathField;
+    StringBuilder partitionPath = new StringBuilder();
+
+    //Corresponds to no partition case
+    if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {
+      return "";
+    }
+    for (String field : partitionPathFields) {
+      String[] fieldWithType = field.split(SPLIT_REGEX);
+      if (fieldWithType.length != 2) {
+        throw new HoodieKeyException("Unable to find field names for partition path in proper format");
+      }
+
+      partitionPathField = fieldWithType[0];
+      PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
+      switch (keyType) {
+        case SIMPLE:
+          SimpleKeyGenerator simpleKeyGenerator = new SimpleKeyGenerator(properties);
+          partitionPath.append(simpleKeyGenerator.getPartitionPath(record, partitionPathField));

Review comment:
       can we do it in one line. 
   partitionPath.append(new SimpleKeyGenerator(properties).getPartitionPath(record, partitionPathField));
   

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.keygen;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.config.TypedProperties;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
+ * Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
+ * "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
+ *
+ * properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
+ *
+ * The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
+ *
+ * Few points to consider:
+ * 1. If you want to customise some partition path field on a timestamp basis, you can use field1:timestampBased
+ * 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
+ * 3. If you want your table to be non partitioned, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List<String> recordKeyFields;
+  protected final List<String> partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+    SIMPLE, TIMESTAMP
+  }
+
+  public CustomKeyGenerator(TypedProperties props) {
+    super(props);
+    this.properties = props;
+    this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields =
+      Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+  }
+
+  @Override
+  public HoodieKey getKey(GenericRecord record) {
+    //call function to get the record key
+    String recordKey = getRecordKey(record);
+    //call function to get the partition key based on the type for that partition path field
+    String partitionPath = getPartitionPath(record);
+    return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {
+    if (partitionPathFields == null) {
+      throw new HoodieKeyException("Unable to find field names for partition path in cfg");
+    }
+
+    String partitionPathField;
+    StringBuilder partitionPath = new StringBuilder();
+
+    //Corresponds to no partition case
+    if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {
+      return "";
+    }
+    for (String field : partitionPathFields) {
+      String[] fieldWithType = field.split(SPLIT_REGEX);
+      if (fieldWithType.length != 2) {
+        throw new HoodieKeyException("Unable to find field names for partition path in proper format");
+      }
+
+      partitionPathField = fieldWithType[0];
+      PartitionKeyType keyType = PartitionKeyType.valueOf(fieldWithType[1].toUpperCase());
+      switch (keyType) {
+        case SIMPLE:
+          SimpleKeyGenerator simpleKeyGenerator = new SimpleKeyGenerator(properties);
+          partitionPath.append(simpleKeyGenerator.getPartitionPath(record, partitionPathField));
+          break;
+        case TIMESTAMP:
+          TimestampBasedKeyGenerator timestampBasedKeyGenerator = new TimestampBasedKeyGenerator(properties);
+          partitionPath.append(timestampBasedKeyGenerator.getPartitionPath(record, partitionPathField));

Review comment:
       same here. can we do it in one line. 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.keygen;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.config.TypedProperties;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
+ * Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
+ * "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
+ *
+ * properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
+ *
+ * The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
+ *
+ * Few points to consider:
+ * 1. If you want to customise some partition path field on a timestamp basis, you can use field1:timestampBased
+ * 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
+ * 3. If you want your table to be non partitioned, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List<String> recordKeyFields;
+  protected final List<String> partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+    SIMPLE, TIMESTAMP
+  }
+
+  public CustomKeyGenerator(TypedProperties props) {
+    super(props);
+    this.properties = props;
+    this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields =
+      Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+  }
+
+  @Override
+  public HoodieKey getKey(GenericRecord record) {
+    //call function to get the record key
+    String recordKey = getRecordKey(record);
+    //call function to get the partition key based on the type for that partition path field
+    String partitionPath = getPartitionPath(record);
+    return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {
+    if (partitionPathFields == null) {
+      throw new HoodieKeyException("Unable to find field names for partition path in cfg");
+    }
+
+    String partitionPathField;
+    StringBuilder partitionPath = new StringBuilder();
+
+    //Corresponds to no partition case
+    if (partitionPathFields.size() == 1 && partitionPathFields.get(0).isEmpty()) {

Review comment:
       sorry, if size = 1, how is it a no partition case ? 

##########
File path: hudi-spark/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
##########
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.keygen;
+
+import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.config.TypedProperties;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.exception.HoodieDeltaStreamerException;
+import org.apache.hudi.exception.HoodieKeyException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields.
+ * Similarly partition path can be configured to have multiple fields or only one field. This class expects value for prop
+ * "hoodie.datasource.write.partitionpath.field" in a specific format. For example:
+ *
+ * properties.put("hoodie.datasource.write.partitionpath.field", "field1:PartitionKeyType1,field2:PartitionKeyType2").
+ *
+ * The complete partition path is created as <value for field1 basis PartitionKeyType1>/<value for field2 basis PartitionKeyType2> and so on.
+ *
+ * Few points to consider:
+ * 1. If you want to customise some partition path field on a timestamp basis, you can use field1:timestampBased
+ * 2. If you simply want to have the value of your configured field in the partition path, use field1:simple
+ * 3. If you want your table to be non partitioned, simply leave it as blank.
+ *
+ * RecordKey is internally generated using either SimpleKeyGenerator or ComplexKeyGenerator.
+ */
+public class CustomKeyGenerator extends KeyGenerator {
+
+  protected final List<String> recordKeyFields;
+  protected final List<String> partitionPathFields;
+  protected final TypedProperties properties;
+  private static final String DEFAULT_PARTITION_PATH_SEPARATOR = "/";
+  private static final String SPLIT_REGEX = ":";
+
+  /**
+   * Used as a part of config in CustomKeyGenerator.java.
+   */
+  public enum PartitionKeyType {
+    SIMPLE, TIMESTAMP
+  }
+
+  public CustomKeyGenerator(TypedProperties props) {
+    super(props);
+    this.properties = props;
+    this.recordKeyFields = Arrays.stream(props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields =
+      Arrays.stream(props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()).split(",")).map(String::trim).collect(Collectors.toList());
+  }
+
+  @Override
+  public HoodieKey getKey(GenericRecord record) {
+    //call function to get the record key
+    String recordKey = getRecordKey(record);
+    //call function to get the partition key based on the type for that partition path field
+    String partitionPath = getPartitionPath(record);
+    return new HoodieKey(recordKey, partitionPath);
+  }
+
+  public String getPartitionPath(GenericRecord record) {

Review comment:
       why all these methods are public ? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org